You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2022/03/13 16:55:06 UTC

[GitHub] [pinot] Jackie-Jiang commented on a change in pull request #8273: Add prefixesToRename config for renaming fields upon ingestion

Jackie-Jiang commented on a change in pull request #8273:
URL: https://github.com/apache/pinot/pull/8273#discussion_r825474664



##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
##########
@@ -89,25 +90,30 @@
   private final List<String> _fieldsToUnnest;
   private final String _delimiter;
   private final ComplexTypeConfig.CollectionNotUnnestedToJson _collectionNotUnnestedToJson;
+  private final Map<String, String> _prefixesToRename;
 
   public ComplexTypeTransformer(TableConfig tableConfig) {
-    this(parseFieldsToUnnest(tableConfig), parseDelimiter(tableConfig), parseCollectionNotUnnestedToJson(tableConfig));
+    this(parseFieldsToUnnest(tableConfig), parseDelimiter(tableConfig),
+            parseCollectionNotUnnestedToJson(tableConfig), parsePrefixesToRename(tableConfig));
   }
 
   @VisibleForTesting
   ComplexTypeTransformer(List<String> fieldsToUnnest, String delimiter) {
-    this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE);
+    this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE, new HashMap<>());
   }
 
   @VisibleForTesting
-  ComplexTypeTransformer(List<String> fieldsToUnnest, String delimiter,
-      ComplexTypeConfig.CollectionNotUnnestedToJson collectionNotUnnestedToJson) {
+  ComplexTypeTransformer(List<String> fieldsToUnnest,

Review comment:
       (minor, code format) keep the Pinot Style format

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
##########
@@ -89,25 +90,30 @@
   private final List<String> _fieldsToUnnest;
   private final String _delimiter;
   private final ComplexTypeConfig.CollectionNotUnnestedToJson _collectionNotUnnestedToJson;
+  private final Map<String, String> _prefixesToRename;
 
   public ComplexTypeTransformer(TableConfig tableConfig) {
-    this(parseFieldsToUnnest(tableConfig), parseDelimiter(tableConfig), parseCollectionNotUnnestedToJson(tableConfig));
+    this(parseFieldsToUnnest(tableConfig), parseDelimiter(tableConfig),
+            parseCollectionNotUnnestedToJson(tableConfig), parsePrefixesToRename(tableConfig));
   }
 
   @VisibleForTesting
   ComplexTypeTransformer(List<String> fieldsToUnnest, String delimiter) {
-    this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE);
+    this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE, new HashMap<>());
   }
 
   @VisibleForTesting
-  ComplexTypeTransformer(List<String> fieldsToUnnest, String delimiter,
-      ComplexTypeConfig.CollectionNotUnnestedToJson collectionNotUnnestedToJson) {
+  ComplexTypeTransformer(List<String> fieldsToUnnest,
+                         String delimiter,
+                         ComplexTypeConfig.CollectionNotUnnestedToJson collectionNotUnnestedToJson,
+                         Map<String, String> prefixesToRename) {
     _fieldsToUnnest = new ArrayList<>(fieldsToUnnest);
     _delimiter = delimiter;
     _collectionNotUnnestedToJson = collectionNotUnnestedToJson;
     // the unnest fields are sorted to achieve the topological sort of the collections, so that the parent collection
     // (e.g. foo) is unnested before the child collection (e.g. foo.bar)
     Collections.sort(_fieldsToUnnest);
+    _prefixesToRename = new HashMap<>(prefixesToRename);

Review comment:
       Don't copy a new map since it will create unnecessary garbage

##########
File path: pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/ComplexTypeConfig.java
##########
@@ -44,13 +45,18 @@
   @JsonPropertyDescription("The mode of converting collection to JSON string")
   private final CollectionNotUnnestedToJson _collectionNotUnnestedToJson;
 
+  @JsonPropertyDescription("Map of <prefix, replacement> so matching fields are renamed to start with the replacement")
+  private final Map<String, String> _prefixesToRename;
+
   @JsonCreator
   public ComplexTypeConfig(@JsonProperty("fieldsToUnnest") @Nullable List<String> fieldsToUnnest,
       @JsonProperty("delimiter") @Nullable String delimiter,
-      @JsonProperty("collectionNotUnnestedToJson") @Nullable CollectionNotUnnestedToJson collectionNotUnnestedToJson) {
+      @JsonProperty("collectionNotUnnestedToJson") @Nullable CollectionNotUnnestedToJson collectionNotUnnestedToJson,
+                           @JsonProperty("prefixesToRename") @Nullable Map<String, String> prefixesToRename) {

Review comment:
       (code format) reformat this

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
##########
@@ -89,25 +90,30 @@
   private final List<String> _fieldsToUnnest;
   private final String _delimiter;
   private final ComplexTypeConfig.CollectionNotUnnestedToJson _collectionNotUnnestedToJson;
+  private final Map<String, String> _prefixesToRename;
 
   public ComplexTypeTransformer(TableConfig tableConfig) {
-    this(parseFieldsToUnnest(tableConfig), parseDelimiter(tableConfig), parseCollectionNotUnnestedToJson(tableConfig));
+    this(parseFieldsToUnnest(tableConfig), parseDelimiter(tableConfig),
+            parseCollectionNotUnnestedToJson(tableConfig), parsePrefixesToRename(tableConfig));
   }
 
   @VisibleForTesting
   ComplexTypeTransformer(List<String> fieldsToUnnest, String delimiter) {
-    this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE);
+    this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE, new HashMap<>());

Review comment:
       (minor)
   ```suggestion
       this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE, Collections.emptyMap());
   ```

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
##########
@@ -279,6 +295,31 @@ protected void flattenMap(GenericRow record, List<String> columns) {
     }
   }
 
+  /**
+   * Loops through all columns and renames the column's prefix with the corresponding replacement if the prefix matches.
+   */
+  @VisibleForTesting
+  protected void renamePrefixes(GenericRow record) {
+    if (_prefixesToRename.isEmpty()) {
+      return;
+    }
+    List<String> fields = new ArrayList<>(record.getFieldToValueMap().keySet());

Review comment:
       Avoid creating this extra list

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
##########
@@ -348,6 +349,23 @@ public static void validateIngestionConfig(TableConfig tableConfig, @Nullable Sc
           }
         }
       }
+
+      // Complex configs
+      ComplexTypeConfig complexTypeConfig = ingestionConfig.getComplexTypeConfig();
+      if (complexTypeConfig != null && schema != null) {
+        Map<String, String> prefixesToRename = complexTypeConfig.getPrefixesToRename();
+        Set<String> fieldNames = schema.getFieldSpecMap().keySet();
+        if (prefixesToRename != null) {
+          for (String prefix : prefixesToRename.keySet()) {
+            for (String field : fieldNames) {
+              if (field.startsWith(prefix)) {

Review comment:
       (minor) Use `Preconditions.checkState()` with exception message template to simplify this for readability 

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/utils/TableConfigUtils.java
##########
@@ -348,6 +349,23 @@ public static void validateIngestionConfig(TableConfig tableConfig, @Nullable Sc
           }
         }
       }
+
+      // Complex configs
+      ComplexTypeConfig complexTypeConfig = ingestionConfig.getComplexTypeConfig();
+      if (complexTypeConfig != null && schema != null) {
+        Map<String, String> prefixesToRename = complexTypeConfig.getPrefixesToRename();
+        Set<String> fieldNames = schema.getFieldSpecMap().keySet();
+        if (prefixesToRename != null) {

Review comment:
       ```suggestion
           if (MapUtils.isNotEmpty(prefixesToRename)) {
   ```

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
##########
@@ -149,12 +155,22 @@ public static ComplexTypeTransformer getComplexTypeTransformer(TableConfig table
     }
   }
 
+  private static Map<String, String> parsePrefixesToRename(TableConfig tableConfig) {
+    if (tableConfig.getIngestionConfig() != null && tableConfig.getIngestionConfig().getComplexTypeConfig() != null
+            && tableConfig.getIngestionConfig().getComplexTypeConfig().getPrefixesToRename() != null) {
+      return tableConfig.getIngestionConfig().getComplexTypeConfig().getPrefixesToRename();
+    } else {
+      return new HashMap<>();

Review comment:
       (minor)
   ```suggestion
         return Collections.emptyMap();
   ```

##########
File path: pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
##########
@@ -279,6 +295,31 @@ protected void flattenMap(GenericRow record, List<String> columns) {
     }
   }
 
+  /**
+   * Loops through all columns and renames the column's prefix with the corresponding replacement if the prefix matches.
+   */
+  @VisibleForTesting
+  protected void renamePrefixes(GenericRow record) {
+    if (_prefixesToRename.isEmpty()) {
+      return;
+    }
+    List<String> fields = new ArrayList<>(record.getFieldToValueMap().keySet());
+    for (String prefix : _prefixesToRename.keySet()) {

Review comment:
       Use `.entrySet()` to avoid the extra map lookups on line 312




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org