This is an automated email from the ASF dual-hosted git repository.

yupeng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 16f94a1  Rename some complex type configs (#6953)
16f94a1 is described below

commit 16f94a19a82807244617e7436c7ba57a1ed56164
Author: Yupeng Fu <yupe...@users.noreply.github.com>
AuthorDate: Thu May 20 15:32:34 2021 -0700

    Rename some complex type configs (#6953)
---
 .../common/utils/config/TableConfigSerDeTest.java  |  4 +-
 .../pinot/plugin/inputformat/avro/AvroUtils.java   | 50 +++++++++----------
 .../plugin/inputformat/avro/AvroUtilsTest.java     |  8 ++--
 .../recordtransformer/ComplexTypeTransformer.java  | 56 +++++++++++-----------
 .../ComplexTypeTransformerTest.java                |  4 +-
 .../config/table/ingestion/ComplexTypeConfig.java  | 22 ++++-----
 .../java/org/apache/pinot/spi/utils/JsonUtils.java | 40 ++++++++--------
 .../org/apache/pinot/spi/utils/JsonUtilsTest.java  |  8 ++--
 .../admin/command/AvroSchemaToPinotSchema.java     | 32 ++++++-------
 .../tools/admin/command/JsonToPinotSchema.java     | 34 ++++++-------
 ...s_offline_complexTypeHandling_table_config.json |  2 +-
 ...eHandling_meetupRsvp_realtime_table_config.json |  2 +-
 12 files changed, 131 insertions(+), 131 deletions(-)

diff --git 
a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
 
b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
index 978faf0..c151b81 100644
--- 
a/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
+++ 
b/pinot-common/src/test/java/org/apache/pinot/common/utils/config/TableConfigSerDeTest.java
@@ -274,11 +274,11 @@ public class TableConfigSerDeTest {
       streamConfigMaps.add(streamConfigMap);
       List<Map<String, String>> batchConfigMaps = new ArrayList<>();
       batchConfigMaps.add(batchConfigMap);
-      List<String> unnestFields = Arrays.asList("c1, c2");
+      List<String> fieldsToUnnest = Arrays.asList("c1, c2");
       IngestionConfig ingestionConfig =
           new IngestionConfig(new BatchIngestionConfig(batchConfigMaps, 
"APPEND", "HOURLY"),
               new StreamIngestionConfig(streamConfigMaps), new 
FilterConfig("filterFunc(foo)"), transformConfigs,
-              new ComplexTypeConfig(unnestFields, ".", 
ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE));
+              new ComplexTypeConfig(fieldsToUnnest, ".", 
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE));
       TableConfig tableConfig = 
tableConfigBuilder.setIngestionConfig(ingestionConfig).build();
 
       checkIngestionConfig(tableConfig);
diff --git 
a/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroUtils.java
 
b/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroUtils.java
index ae72c85..16674ad 100644
--- 
a/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroUtils.java
+++ 
b/pinot-plugins/pinot-input-format/pinot-avro-base/src/main/java/org/apache/pinot/plugin/inputformat/avro/AvroUtils.java
@@ -80,20 +80,20 @@ public class AvroUtils {
    * @param avroSchema Avro schema
    * @param fieldTypeMap Map from column to field type
    * @param timeUnit Time unit
-   * @param unnestFields the fields to unnest
+   * @param fieldsToUnnest the fields to unnest
    * @param delimiter the delimiter to separate components in nested structure
-   * @param collectionToJsonMode the mode of converting collection to JSON
+   * @param collectionNotUnnestedToJson the mode of converting collection to 
JSON
    *
    * @return Pinot schema
    */
   public static Schema 
getPinotSchemaFromAvroSchemaWithComplexTypeHandling(org.apache.avro.Schema 
avroSchema,
-      @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit, List<String> unnestFields,
-      String delimiter, ComplexTypeConfig.CollectionToJsonMode 
collectionToJsonMode) {
+      @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit, List<String> fieldsToUnnest,
+      String delimiter, ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson) {
     Schema pinotSchema = new Schema();
 
     for (Field field : avroSchema.getFields()) {
-      extractSchemaWithComplexTypeHandling(field.schema(), unnestFields, 
delimiter, field.name(), pinotSchema,
-          fieldTypeMap, timeUnit, collectionToJsonMode);
+      extractSchemaWithComplexTypeHandling(field.schema(), fieldsToUnnest, 
delimiter, field.name(), pinotSchema,
+          fieldTypeMap, timeUnit, collectionNotUnnestedToJson);
     }
     return pinotSchema;
   }
@@ -134,21 +134,21 @@ public class AvroUtils {
    * @param fieldTypeMap Map from column to field type
    * @param timeUnit Time unit
    * @param complexType if allows complex-type handling
-   * @param unnestFields the fields to unnest
+   * @param fieldsToUnnest the fields to unnest
    * @param delimiter the delimiter separating components in nested structure
-   * @param collectionToJsonMode to mode of converting collection to JSON 
string
+   * @param collectionNotUnnestedToJson to mode of converting collection to 
JSON string
    * @return Pinot schema
    */
   public static Schema getPinotSchemaFromAvroSchemaFile(File avroSchemaFile,
       @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit, boolean complexType,
-      List<String> unnestFields, String delimiter, 
ComplexTypeConfig.CollectionToJsonMode collectionToJsonMode)
+      List<String> fieldsToUnnest, String delimiter, 
ComplexTypeConfig.CollectionNotUnnestedToJson collectionNotUnnestedToJson)
       throws IOException {
     org.apache.avro.Schema avroSchema = new 
org.apache.avro.Schema.Parser().parse(avroSchemaFile);
     if (!complexType) {
       return getPinotSchemaFromAvroSchema(avroSchema, fieldTypeMap, timeUnit);
     } else {
-      return getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema, 
fieldTypeMap, timeUnit, unnestFields,
-          delimiter, collectionToJsonMode);
+      return getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema, 
fieldTypeMap, timeUnit, fieldsToUnnest,
+          delimiter, collectionNotUnnestedToJson);
     }
   }
 
@@ -285,9 +285,9 @@ public class AvroUtils {
   }
 
   private static void 
extractSchemaWithComplexTypeHandling(org.apache.avro.Schema fieldSchema,
-      List<String> unnestFields, String delimiter, String path, Schema 
pinotSchema,
+      List<String> fieldsToUnnest, String delimiter, String path, Schema 
pinotSchema,
       @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit,
-      ComplexTypeConfig.CollectionToJsonMode collectionToJsonMode) {
+      ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson) {
     org.apache.avro.Schema.Type fieldType = fieldSchema.getType();
     switch (fieldType) {
       case UNION:
@@ -302,29 +302,29 @@ public class AvroUtils {
           }
         }
         if (nonNullSchema != null) {
-          extractSchemaWithComplexTypeHandling(nonNullSchema, unnestFields, 
delimiter, path, pinotSchema, fieldTypeMap,
-              timeUnit, collectionToJsonMode);
+          extractSchemaWithComplexTypeHandling(nonNullSchema, fieldsToUnnest, 
delimiter, path, pinotSchema, fieldTypeMap,
+              timeUnit, collectionNotUnnestedToJson);
         } else {
           throw new IllegalStateException("Cannot find non-null schema in 
UNION schema");
         }
         break;
       case RECORD:
         for (Field innerField : fieldSchema.getFields()) {
-          extractSchemaWithComplexTypeHandling(innerField.schema(), 
unnestFields, delimiter,
+          extractSchemaWithComplexTypeHandling(innerField.schema(), 
fieldsToUnnest, delimiter,
               String.join(delimiter, path, innerField.name()), pinotSchema, 
fieldTypeMap, timeUnit,
-              collectionToJsonMode);
+              collectionNotUnnestedToJson);
         }
         break;
       case ARRAY:
         org.apache.avro.Schema elementType = fieldSchema.getElementType();
-        if (unnestFields.contains(path)) {
-          extractSchemaWithComplexTypeHandling(elementType, unnestFields, 
delimiter, path, pinotSchema, fieldTypeMap,
-              timeUnit, collectionToJsonMode);
-        } else if (collectionToJsonMode == 
ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE && AvroSchemaUtil
+        if (fieldsToUnnest.contains(path)) {
+          extractSchemaWithComplexTypeHandling(elementType, fieldsToUnnest, 
delimiter, path, pinotSchema, fieldTypeMap,
+              timeUnit, collectionNotUnnestedToJson);
+        } else if (collectionNotUnnestedToJson == 
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE && AvroSchemaUtil
             .isPrimitiveType(elementType.getType())) {
           addFieldToPinotSchema(pinotSchema, 
AvroSchemaUtil.valueOf(elementType.getType()), path, false, fieldTypeMap,
               timeUnit);
-        } else if (shallConvertToJson(collectionToJsonMode, elementType)) {
+        } else if (shallConvertToJson(collectionNotUnnestedToJson, 
elementType)) {
           addFieldToPinotSchema(pinotSchema, DataType.STRING, path, true, 
fieldTypeMap, timeUnit);
         }
         // do not include the node for other cases
@@ -335,9 +335,9 @@ public class AvroUtils {
     }
   }
 
-  private static boolean 
shallConvertToJson(ComplexTypeConfig.CollectionToJsonMode collectionToJsonMode,
+  private static boolean 
shallConvertToJson(ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson,
       org.apache.avro.Schema elementType) {
-    switch (collectionToJsonMode) {
+    switch (collectionNotUnnestedToJson) {
       case ALL:
         return true;
       case NONE:
@@ -345,7 +345,7 @@ public class AvroUtils {
       case NON_PRIMITIVE:
         return !AvroSchemaUtil.isPrimitiveType(elementType.getType());
       default:
-        throw new IllegalArgumentException(String.format("Unsupported 
collectionToJsonMode %s", collectionToJsonMode));
+        throw new IllegalArgumentException(String.format("Unsupported 
collectionNotUnnestedToJson %s", collectionNotUnnestedToJson));
     }
   }
 
diff --git 
a/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java
 
b/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java
index e0140ea..bbe7c59 100644
--- 
a/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java
+++ 
b/pinot-plugins/pinot-input-format/pinot-avro-base/src/test/java/org/apache/pinot/plugin/inputformat/avro/AvroUtilsTest.java
@@ -90,7 +90,7 @@ public class AvroUtilsTest {
             .put("hoursSinceEpoch", FieldType.TIME).put("m1", 
FieldType.METRIC).build();
     Schema inferredPinotSchema = AvroUtils
         .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema, 
fieldSpecMap, TimeUnit.HOURS,
-            new ArrayList<>(), ".", 
ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE);
+            new ArrayList<>(), ".", 
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
     Schema expectedSchema =
         new Schema.SchemaBuilder().addSingleValueDimension("d1", 
DataType.STRING).addMetric("m1", DataType.INT)
             .addSingleValueDimension("tuple.streetaddress", DataType.STRING)
@@ -102,7 +102,7 @@ public class AvroUtilsTest {
     // unnest collection entries
     inferredPinotSchema = AvroUtils
         .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema, 
fieldSpecMap, TimeUnit.HOURS,
-            Lists.newArrayList("entries"), ".", 
ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE);
+            Lists.newArrayList("entries"), ".", 
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
     expectedSchema =
         new Schema.SchemaBuilder().addSingleValueDimension("d1", 
DataType.STRING).addMetric("m1", DataType.INT)
             .addSingleValueDimension("tuple.streetaddress", DataType.STRING)
@@ -114,7 +114,7 @@ public class AvroUtilsTest {
     // change delimiter
     inferredPinotSchema = AvroUtils
         .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema, 
fieldSpecMap, TimeUnit.HOURS,
-            Lists.newArrayList(), "_", 
ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE);
+            Lists.newArrayList(), "_", 
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
     expectedSchema =
         new Schema.SchemaBuilder().addSingleValueDimension("d1", 
DataType.STRING).addMetric("m1", DataType.INT)
             .addSingleValueDimension("tuple_streetaddress", DataType.STRING)
@@ -126,7 +126,7 @@ public class AvroUtilsTest {
     // change the handling of collection-to-json option, d2 will become string
     inferredPinotSchema = AvroUtils
         .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema, 
fieldSpecMap, TimeUnit.HOURS,
-            Lists.newArrayList("entries"), ".", 
ComplexTypeConfig.CollectionToJsonMode.ALL);
+            Lists.newArrayList("entries"), ".", 
ComplexTypeConfig.CollectionNotUnnestedToJson.ALL);
     expectedSchema =
         new Schema.SchemaBuilder().addSingleValueDimension("d1", 
DataType.STRING).addMetric("m1", DataType.INT)
             .addSingleValueDimension("tuple.streetaddress", DataType.STRING)
diff --git 
a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
 
b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
index e843a25..4c8a046 100644
--- 
a/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
+++ 
b/pinot-segment-local/src/main/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformer.java
@@ -81,36 +81,36 @@ import org.apache.pinot.spi.data.readers.GenericRow;
  */
 public class ComplexTypeTransformer implements RecordTransformer {
   public static final String DEFAULT_DELIMITER = ".";
-  public static final ComplexTypeConfig.CollectionToJsonMode 
DEFAULT_COLLECTION_TO_JSON_MODE =
-      ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE;
-  private final List<String> _unnestFields;
+  public static final ComplexTypeConfig.CollectionNotUnnestedToJson 
DEFAULT_COLLECTION_TO_JSON_MODE =
+      ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE;
+  private final List<String> _fieldsToUnnest;
   private final String _delimiter;
-  private final ComplexTypeConfig.CollectionToJsonMode _collectionToJsonMode;
+  private final ComplexTypeConfig.CollectionNotUnnestedToJson 
_collectionNotUnnestedToJson;
 
   public ComplexTypeTransformer(TableConfig tableConfig) {
-    this(parseUnnestFields(tableConfig), parseDelimiter(tableConfig), 
parseCollectionToJsonMode(tableConfig));
+    this(parseFieldsToUnnest(tableConfig), parseDelimiter(tableConfig), 
parseCollectionNotUnnestedToJson(tableConfig));
   }
 
   @VisibleForTesting
-  ComplexTypeTransformer(List<String> unnestFields, String delimiter) {
-    this(unnestFields, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE);
+  ComplexTypeTransformer(List<String> fieldsToUnnest, String delimiter) {
+    this(fieldsToUnnest, delimiter, DEFAULT_COLLECTION_TO_JSON_MODE);
   }
 
   @VisibleForTesting
-  ComplexTypeTransformer(List<String> unnestFields, String delimiter,
-      ComplexTypeConfig.CollectionToJsonMode collectionToJsonMode) {
-    _unnestFields = new ArrayList<>(unnestFields);
+  ComplexTypeTransformer(List<String> fieldsToUnnest, String delimiter,
+      ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson) {
+    _fieldsToUnnest = new ArrayList<>(fieldsToUnnest);
     _delimiter = delimiter;
-    _collectionToJsonMode = collectionToJsonMode;
+    _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(_unnestFields);
+    Collections.sort(_fieldsToUnnest);
   }
 
-  private static List<String> parseUnnestFields(TableConfig tableConfig) {
+  private static List<String> parseFieldsToUnnest(TableConfig tableConfig) {
     if (tableConfig.getIngestionConfig() != null && 
tableConfig.getIngestionConfig().getComplexTypeConfig() != null
-        && 
tableConfig.getIngestionConfig().getComplexTypeConfig().getUnnestFields() != 
null) {
-      return 
tableConfig.getIngestionConfig().getComplexTypeConfig().getUnnestFields();
+        && 
tableConfig.getIngestionConfig().getComplexTypeConfig().getFieldsToUnnest() != 
null) {
+      return 
tableConfig.getIngestionConfig().getComplexTypeConfig().getFieldsToUnnest();
     } else {
       return new ArrayList<>();
     }
@@ -136,10 +136,10 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
     return null;
   }
 
-  private static ComplexTypeConfig.CollectionToJsonMode 
parseCollectionToJsonMode(TableConfig tableConfig) {
+  private static ComplexTypeConfig.CollectionNotUnnestedToJson 
parseCollectionNotUnnestedToJson(TableConfig tableConfig) {
     if (tableConfig.getIngestionConfig() != null && 
tableConfig.getIngestionConfig().getComplexTypeConfig() != null
-        && 
tableConfig.getIngestionConfig().getComplexTypeConfig().getCollectionToJsonMode()
 != null) {
-      return 
tableConfig.getIngestionConfig().getComplexTypeConfig().getCollectionToJsonMode();
+        && 
tableConfig.getIngestionConfig().getComplexTypeConfig().getCollectionNotUnnestedToJson()
 != null) {
+      return 
tableConfig.getIngestionConfig().getComplexTypeConfig().getCollectionNotUnnestedToJson();
     } else {
       return DEFAULT_COLLECTION_TO_JSON_MODE;
     }
@@ -149,7 +149,7 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
   @Override
   public GenericRow transform(GenericRow record) {
     flattenMap(record, new ArrayList<>(record.getFieldToValueMap().keySet()));
-    for (String collection : _unnestFields) {
+    for (String collection : _fieldsToUnnest) {
       unnestCollection(record, collection);
     }
     return record;
@@ -236,7 +236,7 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
         flattenMap(record, mapColumns);
       } else if (value instanceof Collection) {
         Collection collection = (Collection) value;
-        if (_unnestFields.contains(column)) {
+        if (_fieldsToUnnest.contains(column)) {
           for (Object inner : collection) {
             if (inner instanceof Map) {
               Map<String, Object> innerMap = (Map<String, Object>) inner;
@@ -255,7 +255,7 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
         }
       } else if (isArray(value)) {
         Object[] array = (Object[]) value;
-        if (_unnestFields.contains(column)) {
+        if (_fieldsToUnnest.contains(column)) {
           for (Object inner : array) {
             if (inner instanceof Map) {
               Map<String, Object> innerMap = (Map<String, Object>) inner;
@@ -321,9 +321,9 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
         if (!innerMapFields.isEmpty()) {
           flattenMap(concatName, map, innerMapFields);
         }
-      } else if (value instanceof Collection && 
_unnestFields.contains(concatName)) {
+      } else if (value instanceof Collection && 
_fieldsToUnnest.contains(concatName)) {
         Collection collection = (Collection) value;
-        if (_unnestFields.contains(concatName)) {
+        if (_fieldsToUnnest.contains(concatName)) {
           for (Object inner : (Collection) value) {
             if (inner instanceof Map) {
               Map<String, Object> innerMap = (Map<String, Object>) inner;
@@ -342,7 +342,7 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
         }
       } else if (isArray(value)) {
         Object[] array = (Object[]) value;
-        if (_unnestFields.contains(concatName)) {
+        if (_fieldsToUnnest.contains(concatName)) {
           for (Object inner : (Object[]) value) {
             if (inner instanceof Map) {
               Map<String, Object> innerMap = (Map<String, Object>) inner;
@@ -364,7 +364,7 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
   }
 
   private boolean shallConvertToJson(Object[] value) {
-    switch (_collectionToJsonMode) {
+    switch (_collectionNotUnnestedToJson) {
       case ALL:
         return true;
       case NONE:
@@ -372,12 +372,12 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
       case NON_PRIMITIVE:
         return !containPrimitives(value);
       default:
-        throw new IllegalArgumentException(String.format("Unsupported 
collectionToJsonMode %s", _collectionToJsonMode));
+        throw new IllegalArgumentException(String.format("Unsupported 
collectionNotUnnestedToJson %s", _collectionNotUnnestedToJson));
     }
   }
 
   private boolean shallConvertToJson(Collection value) {
-    switch (_collectionToJsonMode) {
+    switch (_collectionNotUnnestedToJson) {
       case ALL:
         return true;
       case NONE:
@@ -385,7 +385,7 @@ public class ComplexTypeTransformer implements 
RecordTransformer {
       case NON_PRIMITIVE:
         return !containPrimitives(value);
       default:
-        throw new IllegalArgumentException(String.format("Unsupported 
collectionToJsonMode %s", _collectionToJsonMode));
+        throw new IllegalArgumentException(String.format("Unsupported 
collectionNotUnnestedToJson %s", _collectionNotUnnestedToJson));
     }
   }
 
diff --git 
a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformerTest.java
 
b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformerTest.java
index 9124d60..b6243ae 100644
--- 
a/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformerTest.java
+++ 
b/pinot-segment-local/src/test/java/org/apache/pinot/segment/local/recordtransformer/ComplexTypeTransformerTest.java
@@ -296,7 +296,7 @@ public class ComplexTypeTransformerTest {
     // {
     //   "array":"[1,2]"
     // }
-    transformer = new ComplexTypeTransformer(Arrays.asList(), ".", 
ComplexTypeConfig.CollectionToJsonMode.ALL);
+    transformer = new ComplexTypeTransformer(Arrays.asList(), ".", 
ComplexTypeConfig.CollectionNotUnnestedToJson.ALL);
     genericRow = new GenericRow();
     array = new Object[]{1, 2};
     genericRow.putValue("array", array);
@@ -341,7 +341,7 @@ public class ComplexTypeTransformerTest {
     array1[0] = ImmutableMap.of("b", "v1");
     map.put("array1", array1);
     genericRow.putValue("t", map);
-    transformer = new ComplexTypeTransformer(Arrays.asList(), ".", 
ComplexTypeConfig.CollectionToJsonMode.NONE);
+    transformer = new ComplexTypeTransformer(Arrays.asList(), ".", 
ComplexTypeConfig.CollectionNotUnnestedToJson.NONE);
     transformer.transform(genericRow);
     
Assert.assertTrue(ComplexTypeTransformer.isArray(genericRow.getValue("t.array1")));
   }
diff --git 
a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/ComplexTypeConfig.java
 
b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/ComplexTypeConfig.java
index 40505b1..8aaa1f4 100644
--- 
a/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/ComplexTypeConfig.java
+++ 
b/pinot-spi/src/main/java/org/apache/pinot/spi/config/table/ingestion/ComplexTypeConfig.java
@@ -31,31 +31,31 @@ import org.apache.pinot.spi.config.BaseJsonConfig;
  */
 public class ComplexTypeConfig extends BaseJsonConfig {
 
-  public enum CollectionToJsonMode {
+  public enum CollectionNotUnnestedToJson {
     NONE, NON_PRIMITIVE, ALL
   }
 
   @JsonPropertyDescription("The fields to unnest")
-  private final List<String> _unnestFields;
+  private final List<String> _fieldsToUnnest;
 
   @JsonPropertyDescription("The delimiter used to separate components in a 
path")
   private final String _delimiter;
 
   @JsonPropertyDescription("The mode of converting collection to JSON string")
-  private final CollectionToJsonMode _collectionToJsonMode;
+  private final CollectionNotUnnestedToJson _collectionNotUnnestedToJson;
 
   @JsonCreator
-  public ComplexTypeConfig(@JsonProperty("unnestFields") @Nullable 
List<String> unnestFields,
+  public ComplexTypeConfig(@JsonProperty("fieldsToUnnest") @Nullable 
List<String> fieldsToUnnest,
       @JsonProperty("delimiter") @Nullable String delimiter,
-      @JsonProperty("collectionToJsonMode") @Nullable CollectionToJsonMode 
collectionToJsonMode) {
-    _unnestFields = unnestFields;
+      @JsonProperty("collectionNotUnnestedToJson") @Nullable 
CollectionNotUnnestedToJson collectionNotUnnestedToJson) {
+    _fieldsToUnnest = fieldsToUnnest;
     _delimiter = delimiter;
-    _collectionToJsonMode = collectionToJsonMode;
+    _collectionNotUnnestedToJson = collectionNotUnnestedToJson;
   }
 
   @Nullable
-  public List<String> getUnnestFields() {
-    return _unnestFields;
+  public List<String> getFieldsToUnnest() {
+    return _fieldsToUnnest;
   }
 
   @Nullable
@@ -64,7 +64,7 @@ public class ComplexTypeConfig extends BaseJsonConfig {
   }
 
   @Nullable
-  public CollectionToJsonMode getCollectionToJsonMode() {
-    return _collectionToJsonMode;
+  public CollectionNotUnnestedToJson getCollectionNotUnnestedToJson() {
+    return _collectionNotUnnestedToJson;
   }
 }
diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java 
b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
index ecdd6a9..cf40fb9 100644
--- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
+++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/JsonUtils.java
@@ -410,34 +410,34 @@ public class JsonUtils {
 
   public static Schema getPinotSchemaFromJsonFile(File jsonFile,
       @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit,
-      @Nullable List<String> unnestFields, String delimiter,
-      ComplexTypeConfig.CollectionToJsonMode collectionToJsonMode)
+      @Nullable List<String> fieldsToUnnest, String delimiter,
+      ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson)
       throws IOException {
     JsonNode jsonNode = fileToFirstJsonNode(jsonFile);
-    if (unnestFields == null) {
-      unnestFields = new ArrayList<>();
+    if (fieldsToUnnest == null) {
+      fieldsToUnnest = new ArrayList<>();
     }
     Preconditions.checkState(jsonNode.isObject(), "the JSON data shall be an 
object");
-    return getPinotSchemaFromJsonNode(jsonNode, fieldTypeMap, timeUnit, 
unnestFields, delimiter, collectionToJsonMode);
+    return getPinotSchemaFromJsonNode(jsonNode, fieldTypeMap, timeUnit, 
fieldsToUnnest, delimiter, collectionNotUnnestedToJson);
   }
 
   public static Schema getPinotSchemaFromJsonNode(JsonNode jsonNode,
-      @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit, List<String> unnestFields,
-      String delimiter, ComplexTypeConfig.CollectionToJsonMode 
collectionToJsonMode) {
+      @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit, List<String> fieldsToUnnest,
+      String delimiter, ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson) {
     Schema pinotSchema = new Schema();
     Iterator<Map.Entry<String, JsonNode>> fieldIterator = jsonNode.fields();
     while (fieldIterator.hasNext()) {
       Map.Entry<String, JsonNode> fieldEntry = fieldIterator.next();
       JsonNode childNode = fieldEntry.getValue();
-      inferPinotSchemaFromJsonNode(childNode, pinotSchema, 
fieldEntry.getKey(), fieldTypeMap, timeUnit, unnestFields,
-          delimiter, collectionToJsonMode);
+      inferPinotSchemaFromJsonNode(childNode, pinotSchema, 
fieldEntry.getKey(), fieldTypeMap, timeUnit, fieldsToUnnest,
+          delimiter, collectionNotUnnestedToJson);
     }
     return pinotSchema;
   }
 
   private static void inferPinotSchemaFromJsonNode(JsonNode jsonNode, Schema 
pinotSchema, String path,
-      @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit, List<String> unnestFields,
-      String delimiter, ComplexTypeConfig.CollectionToJsonMode 
collectionToJsonMode) {
+      @Nullable Map<String, FieldSpec.FieldType> fieldTypeMap, @Nullable 
TimeUnit timeUnit, List<String> fieldsToUnnest,
+      String delimiter, ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson) {
     if (jsonNode.isNull()) {
       // do nothing
       return;
@@ -452,12 +452,12 @@ public class JsonUtils {
       }
       JsonNode childNode = jsonNode.get(0);
 
-      if (unnestFields.contains(path)) {
-        inferPinotSchemaFromJsonNode(childNode, pinotSchema, path, 
fieldTypeMap, timeUnit, unnestFields, delimiter,
-            collectionToJsonMode);
-      } else if (shallConvertToJson(collectionToJsonMode, childNode)) {
+      if (fieldsToUnnest.contains(path)) {
+        inferPinotSchemaFromJsonNode(childNode, pinotSchema, path, 
fieldTypeMap, timeUnit, fieldsToUnnest, delimiter,
+            collectionNotUnnestedToJson);
+      } else if (shallConvertToJson(collectionNotUnnestedToJson, childNode)) {
         addFieldToPinotSchema(pinotSchema, DataType.STRING, path, true, 
fieldTypeMap, timeUnit);
-      } else if (collectionToJsonMode == 
ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE && childNode
+      } else if (collectionNotUnnestedToJson == 
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE && childNode
           .isValueNode()) {
         addFieldToPinotSchema(pinotSchema, valueOf(childNode), path, false, 
fieldTypeMap, timeUnit);
       }
@@ -468,16 +468,16 @@ public class JsonUtils {
         Map.Entry<String, JsonNode> fieldEntry = fieldIterator.next();
         JsonNode childNode = fieldEntry.getValue();
         inferPinotSchemaFromJsonNode(childNode, pinotSchema, 
String.join(delimiter, path, fieldEntry.getKey()),
-            fieldTypeMap, timeUnit, unnestFields, delimiter, 
collectionToJsonMode);
+            fieldTypeMap, timeUnit, fieldsToUnnest, delimiter, 
collectionNotUnnestedToJson);
       }
     } else {
       throw new IllegalArgumentException(String.format("Unsupported json node 
type", jsonNode.getClass()));
     }
   }
 
-  private static boolean 
shallConvertToJson(ComplexTypeConfig.CollectionToJsonMode collectionToJsonMode,
+  private static boolean 
shallConvertToJson(ComplexTypeConfig.CollectionNotUnnestedToJson 
collectionNotUnnestedToJson,
       JsonNode childNode) {
-    switch (collectionToJsonMode) {
+    switch (collectionNotUnnestedToJson) {
       case ALL:
         return true;
       case NONE:
@@ -485,7 +485,7 @@ public class JsonUtils {
       case NON_PRIMITIVE:
         return !childNode.isValueNode();
       default:
-        throw new IllegalArgumentException(String.format("Unsupported 
collectionToJsonMode %s", collectionToJsonMode));
+        throw new IllegalArgumentException(String.format("Unsupported 
collectionNotUnnestedToJson %s", collectionNotUnnestedToJson));
     }
   }
 
diff --git 
a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java 
b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java
index 4c2167c..7a72875 100644
--- a/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java
+++ b/pinot-spi/src/test/java/org/apache/pinot/spi/utils/JsonUtilsTest.java
@@ -276,7 +276,7 @@ public class JsonUtilsTest {
             .put("hoursSinceEpoch", FieldSpec.FieldType.DATE_TIME).put("m1", 
FieldSpec.FieldType.METRIC).build();
     Schema inferredPinotSchema = JsonUtils
         .getPinotSchemaFromJsonFile(file, fieldSpecMap, TimeUnit.HOURS, new 
ArrayList<>(), ".",
-            ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE);
+            ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
     Schema expectedSchema = new 
Schema.SchemaBuilder().addSingleValueDimension("d1", FieldSpec.DataType.STRING)
         .addMetric("m1", FieldSpec.DataType.INT)
         .addSingleValueDimension("tuple.address.streetaddress", 
FieldSpec.DataType.STRING)
@@ -289,7 +289,7 @@ public class JsonUtilsTest {
     // unnest collection entries
     inferredPinotSchema = JsonUtils
         .getPinotSchemaFromJsonFile(file, fieldSpecMap, TimeUnit.HOURS, 
Lists.newArrayList("entries"), ".",
-            ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE);
+            ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
     expectedSchema = new Schema.SchemaBuilder().addSingleValueDimension("d1", 
FieldSpec.DataType.STRING)
         .addMetric("m1", FieldSpec.DataType.INT)
         .addSingleValueDimension("tuple.address.streetaddress", 
FieldSpec.DataType.STRING)
@@ -303,7 +303,7 @@ public class JsonUtilsTest {
     // change delimiter
     inferredPinotSchema = JsonUtils
         .getPinotSchemaFromJsonFile(file, fieldSpecMap, TimeUnit.HOURS, 
Lists.newArrayList(""), "_",
-            ComplexTypeConfig.CollectionToJsonMode.NON_PRIMITIVE);
+            ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
     expectedSchema = new Schema.SchemaBuilder().addSingleValueDimension("d1", 
FieldSpec.DataType.STRING)
         .addMetric("m1", FieldSpec.DataType.INT)
         .addSingleValueDimension("tuple_address_streetaddress", 
FieldSpec.DataType.STRING)
@@ -316,7 +316,7 @@ public class JsonUtilsTest {
     // change the handling of collection-to-json option, d2 will become string
     inferredPinotSchema = JsonUtils
         .getPinotSchemaFromJsonFile(file, fieldSpecMap, TimeUnit.HOURS, 
Lists.newArrayList("entries"), ".",
-            ComplexTypeConfig.CollectionToJsonMode.ALL);
+            ComplexTypeConfig.CollectionNotUnnestedToJson.ALL);
     expectedSchema = new Schema.SchemaBuilder().addSingleValueDimension("d1", 
FieldSpec.DataType.STRING)
         .addMetric("m1", FieldSpec.DataType.INT)
         .addSingleValueDimension("tuple.address.streetaddress", 
FieldSpec.DataType.STRING)
diff --git 
a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/AvroSchemaToPinotSchema.java
 
b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/AvroSchemaToPinotSchema.java
index 34ea580..82dcf94 100644
--- 
a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/AvroSchemaToPinotSchema.java
+++ 
b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/AvroSchemaToPinotSchema.java
@@ -68,8 +68,8 @@ public class AvroSchemaToPinotSchema extends 
AbstractBaseAdminCommand implements
   @Option(name = "-timeUnit", metaVar = "<string>", usage = "Unit of the time 
column (default DAYS).")
   TimeUnit _timeUnit = TimeUnit.DAYS;
 
-  @Option(name = "-unnestFields", metaVar = "<string>", usage = "Comma 
separated fields to unnest")
-  String _unnestFields;
+  @Option(name = "-fieldsToUnnest", metaVar = "<string>", usage = "Comma 
separated fields to unnest")
+  String _fieldsToUnnest;
 
   @Option(name = "-delimiter", metaVar = "<string>", usage = "The delimiter 
separating components in nested structure, default to dot")
   String _delimiter;
@@ -77,8 +77,8 @@ public class AvroSchemaToPinotSchema extends 
AbstractBaseAdminCommand implements
   @Option(name = "-complexType", metaVar = "<boolean>", usage = "allow 
complex-type handling, default to false")
   boolean _complexType;
 
-  @Option(name = "-collectionToJsonMode", metaVar = "<string>", usage = "The 
mode of converting collection to JSON string, can be NONE/NON_PRIMITIVE/ALL")
-  String _collectionToJsonMode;
+  @Option(name = "-collectionNotUnnestedToJson", metaVar = "<string>", usage = 
"The mode of converting collection to JSON string, can be 
NONE/NON_PRIMITIVE/ALL")
+  String _collectionNotUnnestedToJson;
 
   @SuppressWarnings("FieldCanBeLocal")
   @Option(name = "-help", help = true, aliases = {"-h", "--h", "--help"}, 
usage = "Print this message.")
@@ -97,7 +97,7 @@ public class AvroSchemaToPinotSchema extends 
AbstractBaseAdminCommand implements
     if (_avroSchemaFile != null) {
       schema = AvroUtils
           .getPinotSchemaFromAvroSchemaFile(new File(_avroSchemaFile), 
buildFieldTypesMap(), _timeUnit, _complexType,
-              buildUnnestFields(), getDelimiter(), getCollectionToJsonMode());
+              buildfieldsToUnnest(), getDelimiter(), 
getcollectionNotUnnestedToJson());
     } else if (_avroDataFile != null) {
       schema = AvroUtils.getPinotSchemaFromAvroDataFile(new 
File(_avroDataFile), buildFieldTypesMap(), _timeUnit);
     } else {
@@ -137,8 +137,8 @@ public class AvroSchemaToPinotSchema extends 
AbstractBaseAdminCommand implements
     return "AvroSchemaToPinotSchema -avroSchemaFile " + _avroSchemaFile + " 
-avroDataFile " + _avroDataFile
         + " -outputDir " + _outputDir + " -pinotSchemaName " + 
_pinotSchemaName + " -dimensions " + _dimensions
         + " -metrics " + _metrics + " -timeColumnName " + _timeColumnName + " 
-timeUnit " + _timeUnit
-        + " _unnestFields " + _unnestFields + " _delimiter " + _delimiter + " 
_complexType " + _complexType
-        + " _collectionToJsonMode " + _collectionToJsonMode;
+        + " _fieldsToUnnest " + _fieldsToUnnest + " _delimiter " + _delimiter 
+ " _complexType " + _complexType
+        + " _collectionNotUnnestedToJson " + _collectionNotUnnestedToJson;
   }
 
   /**
@@ -165,21 +165,21 @@ public class AvroSchemaToPinotSchema extends 
AbstractBaseAdminCommand implements
     return fieldTypes;
   }
 
-  private List<String> buildUnnestFields() {
-    List<String> unnestFields = new ArrayList<>();
-    if (_unnestFields != null) {
-      for (String field : _unnestFields.split(",")) {
-        unnestFields.add(field);
+  private List<String> buildfieldsToUnnest() {
+    List<String> fieldsToUnnest = new ArrayList<>();
+    if (_fieldsToUnnest != null) {
+      for (String field : _fieldsToUnnest.split(",")) {
+        fieldsToUnnest.add(field);
       }
     }
-    return unnestFields;
+    return fieldsToUnnest;
   }
 
-  private ComplexTypeConfig.CollectionToJsonMode getCollectionToJsonMode() {
-    if (_collectionToJsonMode == null) {
+  private ComplexTypeConfig.CollectionNotUnnestedToJson 
getcollectionNotUnnestedToJson() {
+    if (_collectionNotUnnestedToJson == null) {
       return ComplexTypeTransformer.DEFAULT_COLLECTION_TO_JSON_MODE;
     }
-    return 
ComplexTypeConfig.CollectionToJsonMode.valueOf(_collectionToJsonMode);
+    return 
ComplexTypeConfig.CollectionNotUnnestedToJson.valueOf(_collectionNotUnnestedToJson);
   }
 
   private String getDelimiter() {
diff --git 
a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/JsonToPinotSchema.java
 
b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/JsonToPinotSchema.java
index 0a63bfc..50c130e 100644
--- 
a/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/JsonToPinotSchema.java
+++ 
b/pinot-tools/src/main/java/org/apache/pinot/tools/admin/command/JsonToPinotSchema.java
@@ -65,14 +65,14 @@ public class JsonToPinotSchema extends 
AbstractBaseAdminCommand implements Comma
   @Option(name = "-timeUnit", metaVar = "<string>", usage = "Unit of the time 
column (default DAYS).")
   TimeUnit _timeUnit = TimeUnit.DAYS;
 
-  @Option(name = "-unnestFields", metaVar = "<string>", usage = "Comma 
separated fields to unnest")
-  String _unnestFields;
+  @Option(name = "-fieldsToUnnest", metaVar = "<string>", usage = "Comma 
separated fields to unnest")
+  String _fieldsToUnnest;
 
   @Option(name = "-delimiter", metaVar = "<string>", usage = "The delimiter 
separating components in nested structure, default to dot")
   String _delimiter;
 
-  @Option(name = "-collectionToJsonMode", metaVar = "<string>", usage = "The 
mode of converting collection to JSON string, can be NONE/NON_PRIMITIVE/ALL")
-  String _collectionToJsonMode;
+  @Option(name = "-collectionNotUnnestedToJson", metaVar = "<string>", usage = 
"The mode of converting collection to JSON string, can be 
NONE/NON_PRIMITIVE/ALL")
+  String _collectionNotUnnestedToJson;
 
   @SuppressWarnings("FieldCanBeLocal")
   @Option(name = "-help", help = true, aliases = {"-h", "--h", "--help"}, 
usage = "Print this message.")
@@ -89,8 +89,8 @@ public class JsonToPinotSchema extends 
AbstractBaseAdminCommand implements Comma
 
     Schema schema;
     schema = JsonUtils
-        .getPinotSchemaFromJsonFile(new File(_jsonFile), buildFieldTypesMap(), 
_timeUnit, buildUnnestFields(),
-            getDelimiter(), getCollectionToJsonMode());
+        .getPinotSchemaFromJsonFile(new File(_jsonFile), buildFieldTypesMap(), 
_timeUnit, buildfieldsToUnnest(),
+            getDelimiter(), getcollectionNotUnnestedToJson());
     schema.setSchemaName(_pinotSchemaName);
 
     File outputDir = new File(_outputDir);
@@ -122,8 +122,8 @@ public class JsonToPinotSchema extends 
AbstractBaseAdminCommand implements Comma
   public String toString() {
     return "JsonToPinotSchema -jsonFile " + _jsonFile + " -outputDir " + 
_outputDir + " -pinotSchemaName "
         + _pinotSchemaName + " -dimensions " + _dimensions + " -metrics " + 
_metrics + " -timeColumnName "
-        + _dateTimeColumnName + " -timeUnit " + _timeUnit + " _unnestFields " 
+ _unnestFields + " _delimiter "
-        + _delimiter + " _collectionToJsonMode " + _collectionToJsonMode;
+        + _dateTimeColumnName + " -timeUnit " + _timeUnit + " _fieldsToUnnest 
" + _fieldsToUnnest + " _delimiter "
+        + _delimiter + " _collectionNotUnnestedToJson " + 
_collectionNotUnnestedToJson;
   }
 
   /**
@@ -150,21 +150,21 @@ public class JsonToPinotSchema extends 
AbstractBaseAdminCommand implements Comma
     return fieldTypes;
   }
 
-  private List<String> buildUnnestFields() {
-    List<String> unnestFields = new ArrayList<>();
-    if (_unnestFields != null) {
-      for (String field : _unnestFields.split(",")) {
-        unnestFields.add(field.trim());
+  private List<String> buildfieldsToUnnest() {
+    List<String> fieldsToUnnest = new ArrayList<>();
+    if (_fieldsToUnnest != null) {
+      for (String field : _fieldsToUnnest.split(",")) {
+        fieldsToUnnest.add(field.trim());
       }
     }
-    return unnestFields;
+    return fieldsToUnnest;
   }
 
-  private ComplexTypeConfig.CollectionToJsonMode getCollectionToJsonMode() {
-    if (_collectionToJsonMode == null) {
+  private ComplexTypeConfig.CollectionNotUnnestedToJson 
getcollectionNotUnnestedToJson() {
+    if (_collectionNotUnnestedToJson == null) {
       return ComplexTypeTransformer.DEFAULT_COLLECTION_TO_JSON_MODE;
     }
-    return 
ComplexTypeConfig.CollectionToJsonMode.valueOf(_collectionToJsonMode);
+    return 
ComplexTypeConfig.CollectionNotUnnestedToJson.valueOf(_collectionNotUnnestedToJson);
   }
 
   private String getDelimiter() {
diff --git 
a/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_complexTypeHandling_table_config.json
 
b/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_complexTypeHandling_table_config.json
index c72cf3c..f59abab 100644
--- 
a/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_complexTypeHandling_table_config.json
+++ 
b/pinot-tools/src/main/resources/examples/batch/githubEvents/githubEvents_offline_complexTypeHandling_table_config.json
@@ -19,7 +19,7 @@
       }
     ],
     "complexTypeConfig": {
-      "unnestFields": ["payload.commits"]
+      "fieldsToUnnest": ["payload.commits"]
     }
   },
   "metadata": {
diff --git 
a/pinot-tools/src/main/resources/examples/stream/meetupRsvp/complexTypeHandling_meetupRsvp_realtime_table_config.json
 
b/pinot-tools/src/main/resources/examples/stream/meetupRsvp/complexTypeHandling_meetupRsvp_realtime_table_config.json
index 6b787a0..2f6cb15 100644
--- 
a/pinot-tools/src/main/resources/examples/stream/meetupRsvp/complexTypeHandling_meetupRsvp_realtime_table_config.json
+++ 
b/pinot-tools/src/main/resources/examples/stream/meetupRsvp/complexTypeHandling_meetupRsvp_realtime_table_config.json
@@ -28,7 +28,7 @@
     "transformConfigs": [
     ],
     "complexTypeConfig": {
-      "unnestFields": ["group.group_topics"]
+      "fieldsToUnnest": ["group.group_topics"]
     }
   },
   "tableIndexConfig": {

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

Reply via email to