This is an automated email from the ASF dual-hosted git repository.
jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git
The following commit(s) were added to refs/heads/master by this push:
new 004c8af Make JsonToPinotSchema consistent with
AvroSchemaToPinotSchema (#7366)
004c8af is described below
commit 004c8af5069957151ac2675dc19da464e65b0eb2
Author: Xiaotian (Jackie) Jiang <[email protected]>
AuthorDate: Wed Aug 25 13:27:32 2021 -0700
Make JsonToPinotSchema consistent with AvroSchemaToPinotSchema (#7366)
Make JsonToPinotSchema tool consistent with AvroSchemaToPinotSchema, and
use -timeColumnName as the option name for DateTimeField.
Remove the deprecated TimeField from the AvroSchemaToPinotSchema.
---
.../pinot/plugin/inputformat/avro/AvroUtils.java | 10 +---
.../plugin/inputformat/avro/AvroUtilsTest.java | 54 +++++++++-------------
.../java/org/apache/pinot/spi/utils/JsonUtils.java | 7 ++-
.../admin/command/AvroSchemaToPinotSchema.java | 38 ++++++++-------
.../tools/admin/command/JsonToPinotSchema.java | 40 ++++++++--------
5 files changed, 63 insertions(+), 86 deletions(-)
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 29cf2ae..bebd903 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
@@ -41,8 +41,6 @@ import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.FieldSpec.DataType;
import org.apache.pinot.spi.data.MetricFieldSpec;
import org.apache.pinot.spi.data.Schema;
-import org.apache.pinot.spi.data.TimeFieldSpec;
-import org.apache.pinot.spi.data.TimeGranularitySpec;
/**
@@ -358,8 +356,7 @@ public class AvroUtils {
if (fieldTypeMap == null) {
pinotSchema.addField(new DimensionFieldSpec(name, dataType,
isSingleValueField));
} else {
- FieldSpec.FieldType fieldType =
- fieldTypeMap.containsKey(name) ? fieldTypeMap.get(name) :
FieldSpec.FieldType.DIMENSION;
+ FieldSpec.FieldType fieldType = fieldTypeMap.getOrDefault(name,
FieldSpec.FieldType.DIMENSION);
Preconditions.checkNotNull(fieldType, "Field type not specified for
field: %s", name);
switch (fieldType) {
case DIMENSION:
@@ -369,11 +366,6 @@ public class AvroUtils {
Preconditions.checkState(isSingleValueField, "Metric field: %s
cannot be multi-valued", name);
pinotSchema.addField(new MetricFieldSpec(name, dataType));
break;
- case TIME:
- Preconditions.checkState(isSingleValueField, "Time field: %s cannot
be multi-valued", name);
- Preconditions.checkNotNull(timeUnit, "Time unit cannot be null");
- pinotSchema.addField(new TimeFieldSpec(new
TimeGranularitySpec(dataType, timeUnit, name)));
- break;
case DATE_TIME:
Preconditions.checkState(isSingleValueField, "Time field: %s cannot
be multi-valued", name);
Preconditions.checkNotNull(timeUnit, "Time unit cannot be null");
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 a3a3372..4bee1d6 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
@@ -28,11 +28,11 @@ import org.apache.pinot.spi.data.FieldSpec;
import org.apache.pinot.spi.data.FieldSpec.DataType;
import org.apache.pinot.spi.data.FieldSpec.FieldType;
import org.apache.pinot.spi.data.Schema;
-import org.apache.pinot.spi.data.TimeGranularitySpec;
-import org.testng.Assert;
import org.testng.annotations.Test;
import org.testng.collections.Lists;
+import static org.testng.Assert.assertEquals;
+
public class AvroUtilsTest {
@@ -49,7 +49,7 @@ public class AvroUtilsTest {
.addSingleValueDimension("d2",
DataType.LONG).addSingleValueDimension("d3", DataType.STRING)
.addSingleValueDimension("m1",
DataType.INT).addSingleValueDimension("m2", DataType.INT)
.addSingleValueDimension("hoursSinceEpoch", DataType.LONG).build();
- Assert.assertEquals(expectedSchema, inferredPinotSchema);
+ assertEquals(expectedSchema, inferredPinotSchema);
}
@Test
@@ -59,24 +59,14 @@ public class AvroUtilsTest {
new
org.apache.avro.Schema.Parser().parse(ClassLoader.getSystemResourceAsStream(AVRO_SCHEMA));
Map<String, FieldSpec.FieldType> fieldSpecMap =
new ImmutableMap.Builder<String, FieldSpec.FieldType>().put("d1",
FieldType.DIMENSION)
- .put("d2", FieldType.DIMENSION).put("d3",
FieldType.DIMENSION).put("hoursSinceEpoch", FieldType.TIME)
+ .put("d2", FieldType.DIMENSION).put("d3",
FieldType.DIMENSION).put("hoursSinceEpoch", FieldType.DATE_TIME)
.put("m1", FieldType.METRIC).put("m2", FieldType.METRIC).build();
Schema inferredPinotSchema =
AvroUtils.getPinotSchemaFromAvroSchema(avroSchema, fieldSpecMap,
TimeUnit.HOURS);
Schema expectedSchema = new
Schema.SchemaBuilder().addSingleValueDimension("d1", DataType.STRING)
.addSingleValueDimension("d2",
DataType.LONG).addSingleValueDimension("d3", DataType.STRING)
.addMetric("m1", DataType.INT).addMetric("m2", DataType.INT)
- .addTime(new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS,
"hoursSinceEpoch"), null).build();
- Assert.assertEquals(expectedSchema, inferredPinotSchema);
-
- fieldSpecMap = new ImmutableMap.Builder<String,
FieldSpec.FieldType>().put("d1", FieldType.DIMENSION)
- .put("d2", FieldType.DIMENSION).put("d3",
FieldType.DIMENSION).put("hoursSinceEpoch", FieldType.DATE_TIME)
- .put("m1", FieldType.METRIC).put("m2", FieldType.METRIC).build();
- inferredPinotSchema = AvroUtils.getPinotSchemaFromAvroSchema(avroSchema,
fieldSpecMap, TimeUnit.HOURS);
- expectedSchema = new Schema.SchemaBuilder().addSingleValueDimension("d1",
DataType.STRING)
- .addSingleValueDimension("d2",
DataType.LONG).addSingleValueDimension("d3", DataType.STRING)
- .addMetric("m1", DataType.INT).addMetric("m2", DataType.INT)
.addDateTime("hoursSinceEpoch", DataType.LONG, "1:HOURS:EPOCH",
"1:HOURS").build();
- Assert.assertEquals(expectedSchema, inferredPinotSchema);
+ assertEquals(inferredPinotSchema, expectedSchema);
}
@Test
@@ -87,45 +77,45 @@ public class AvroUtilsTest {
new
org.apache.avro.Schema.Parser().parse(ClassLoader.getSystemResourceAsStream(AVRO_NESTED_SCHEMA));
Map<String, FieldSpec.FieldType> fieldSpecMap =
new ImmutableMap.Builder<String, FieldSpec.FieldType>().put("d1",
FieldType.DIMENSION)
- .put("hoursSinceEpoch", FieldType.TIME).put("m1",
FieldType.METRIC).build();
- Schema inferredPinotSchema = AvroUtils
- .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
+ .put("hoursSinceEpoch", FieldType.DATE_TIME).put("m1",
FieldType.METRIC).build();
+ Schema inferredPinotSchema =
+
AvroUtils.getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
new ArrayList<>(), ".",
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
Schema expectedSchema =
new Schema.SchemaBuilder().addSingleValueDimension("d1",
DataType.STRING).addMetric("m1", DataType.INT)
.addSingleValueDimension("tuple.streetaddress", DataType.STRING)
.addSingleValueDimension("tuple.city",
DataType.STRING).addSingleValueDimension("entries", DataType.STRING)
.addMultiValueDimension("d2", DataType.INT)
- .addTime(new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS,
"hoursSinceEpoch"), null).build();
- Assert.assertEquals(expectedSchema, inferredPinotSchema);
+ .addDateTime("hoursSinceEpoch", DataType.LONG, "1:HOURS:EPOCH",
"1:HOURS").build();
+ assertEquals(inferredPinotSchema, expectedSchema);
// unnest collection entries
- inferredPinotSchema = AvroUtils
- .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
+ inferredPinotSchema =
+
AvroUtils.getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
Lists.newArrayList("entries"), ".",
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
expectedSchema =
new Schema.SchemaBuilder().addSingleValueDimension("d1",
DataType.STRING).addMetric("m1", DataType.INT)
.addSingleValueDimension("tuple.streetaddress", DataType.STRING)
.addSingleValueDimension("tuple.city",
DataType.STRING).addSingleValueDimension("entries.id", DataType.LONG)
.addSingleValueDimension("entries.description",
DataType.STRING).addMultiValueDimension("d2", DataType.INT)
- .addTime(new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS,
"hoursSinceEpoch"), null).build();
- Assert.assertEquals(expectedSchema, inferredPinotSchema);
+ .addDateTime("hoursSinceEpoch", DataType.LONG, "1:HOURS:EPOCH",
"1:HOURS").build();
+ assertEquals(inferredPinotSchema, expectedSchema);
// change delimiter
- inferredPinotSchema = AvroUtils
- .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
+ inferredPinotSchema =
+
AvroUtils.getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
Lists.newArrayList(), "_",
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE);
expectedSchema =
new Schema.SchemaBuilder().addSingleValueDimension("d1",
DataType.STRING).addMetric("m1", DataType.INT)
.addSingleValueDimension("tuple_streetaddress", DataType.STRING)
.addSingleValueDimension("tuple_city",
DataType.STRING).addSingleValueDimension("entries", DataType.STRING)
.addMultiValueDimension("d2", DataType.INT)
- .addTime(new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS,
"hoursSinceEpoch"), null).build();
- Assert.assertEquals(expectedSchema, inferredPinotSchema);
+ .addDateTime("hoursSinceEpoch", DataType.LONG, "1:HOURS:EPOCH",
"1:HOURS").build();
+ assertEquals(inferredPinotSchema, expectedSchema);
// change the handling of collection-to-json option, d2 will become string
- inferredPinotSchema = AvroUtils
- .getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
+ inferredPinotSchema =
+
AvroUtils.getPinotSchemaFromAvroSchemaWithComplexTypeHandling(avroSchema,
fieldSpecMap, TimeUnit.HOURS,
Lists.newArrayList("entries"), ".",
ComplexTypeConfig.CollectionNotUnnestedToJson.ALL);
expectedSchema =
new Schema.SchemaBuilder().addSingleValueDimension("d1",
DataType.STRING).addMetric("m1", DataType.INT)
@@ -133,7 +123,7 @@ public class AvroUtilsTest {
.addSingleValueDimension("tuple.city",
DataType.STRING).addSingleValueDimension("entries.id", DataType.LONG)
.addSingleValueDimension("entries.description", DataType.STRING)
.addSingleValueDimension("d2", DataType.STRING)
- .addTime(new TimeGranularitySpec(DataType.LONG, TimeUnit.HOURS,
"hoursSinceEpoch"), null).build();
- Assert.assertEquals(expectedSchema, inferredPinotSchema);
+ .addDateTime("hoursSinceEpoch", DataType.LONG, "1:HOURS:EPOCH",
"1:HOURS").build();
+ assertEquals(inferredPinotSchema, expectedSchema);
}
}
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 24a234b..0089085 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
@@ -458,8 +458,8 @@ public class JsonUtils {
collectionNotUnnestedToJson);
} else if (shallConvertToJson(collectionNotUnnestedToJson, childNode)) {
addFieldToPinotSchema(pinotSchema, DataType.STRING, path, true,
fieldTypeMap, timeUnit);
- } else if (collectionNotUnnestedToJson ==
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE && childNode
- .isValueNode()) {
+ } else if (collectionNotUnnestedToJson ==
ComplexTypeConfig.CollectionNotUnnestedToJson.NON_PRIMITIVE
+ && childNode.isValueNode()) {
addFieldToPinotSchema(pinotSchema, valueOf(childNode), path, false,
fieldTypeMap, timeUnit);
}
// do not include the node for other cases
@@ -518,8 +518,7 @@ public class JsonUtils {
if (fieldTypeMap == null) {
pinotSchema.addField(new DimensionFieldSpec(name, dataType,
isSingleValueField));
} else {
- FieldSpec.FieldType fieldType =
- fieldTypeMap.containsKey(name) ? fieldTypeMap.get(name) :
FieldSpec.FieldType.DIMENSION;
+ FieldSpec.FieldType fieldType = fieldTypeMap.getOrDefault(name,
FieldSpec.FieldType.DIMENSION);
Preconditions.checkNotNull(fieldType, "Field type not specified for
field: %s", name);
switch (fieldType) {
case DIMENSION:
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 abbc423..95e51dc 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
@@ -20,7 +20,8 @@ package org.apache.pinot.tools.admin.command;
import java.io.File;
import java.io.FileWriter;
-import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -44,12 +45,12 @@ import org.slf4j.LoggerFactory;
public class AvroSchemaToPinotSchema extends AbstractBaseAdminCommand
implements Command {
private static final Logger LOGGER =
LoggerFactory.getLogger(AvroSchemaToPinotSchema.class);
- @Option(name = "-avroSchemaFile", forbids = {"-avroDataFile"}, metaVar =
"<String>",
- usage = "Path to avro schema file.")
+ @Option(name = "-avroSchemaFile", forbids = {"-avroDataFile"}, metaVar =
"<String>", usage = "Path to avro schema "
+ + "file.")
String _avroSchemaFile;
- @Option(name = "-avroDataFile", forbids = {"-avroSchemaFile"}, metaVar =
"<String>",
- usage = "Path to avro data file.")
+ @Option(name = "-avroDataFile", forbids = {"-avroSchemaFile"}, metaVar =
"<String>", usage = "Path to avro data "
+ + "file.")
String _avroDataFile;
@Option(name = "-outputDir", required = true, metaVar = "<string>", usage =
"Path to output directory")
@@ -73,15 +74,15 @@ public class AvroSchemaToPinotSchema extends
AbstractBaseAdminCommand implements
@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")
+ @Option(name = "-delimiter", metaVar = "<string>", usage = "The delimiter
separating components in nested "
+ + "structure, default to dot")
String _delimiter;
@Option(name = "-complexType", metaVar = "<boolean>", usage = "allow
complex-type handling, default to false")
boolean _complexType;
- @Option(name = "-collectionNotUnnestedToJson", metaVar = "<string>",
- usage = "The mode of converting collection to JSON string, can be
NONE/NON_PRIMITIVE/ALL")
+ @Option(name = "-collectionNotUnnestedToJson", metaVar = "<string>", usage =
"The mode of converting collection to "
+ + "JSON string, can be NONE/NON_PRIMITIVE/ALL")
String _collectionNotUnnestedToJson;
@SuppressWarnings("FieldCanBeLocal")
@@ -99,9 +100,8 @@ public class AvroSchemaToPinotSchema extends
AbstractBaseAdminCommand implements
Schema schema;
if (_avroSchemaFile != null) {
- schema = AvroUtils
- .getPinotSchemaFromAvroSchemaFile(new File(_avroSchemaFile),
buildFieldTypesMap(), _timeUnit, _complexType,
- buildfieldsToUnnest(), getDelimiter(),
getcollectionNotUnnestedToJson());
+ schema = AvroUtils.getPinotSchemaFromAvroSchemaFile(new
File(_avroSchemaFile), buildFieldTypesMap(), _timeUnit,
+ _complexType, buildFieldsToUnnest(), getDelimiter(),
getCollectionNotUnnestedToJson());
} else if (_avroDataFile != null) {
schema = AvroUtils.getPinotSchemaFromAvroDataFile(new
File(_avroDataFile), buildFieldTypesMap(), _timeUnit);
} else {
@@ -164,22 +164,20 @@ public class AvroSchemaToPinotSchema extends
AbstractBaseAdminCommand implements
}
}
if (_timeColumnName != null) {
- fieldTypes.put(_timeColumnName, FieldSpec.FieldType.TIME);
+ fieldTypes.put(_timeColumnName, FieldSpec.FieldType.DATE_TIME);
}
return fieldTypes;
}
- private List<String> buildfieldsToUnnest() {
- List<String> fieldsToUnnest = new ArrayList<>();
+ private List<String> buildFieldsToUnnest() {
if (_fieldsToUnnest != null) {
- for (String field : _fieldsToUnnest.split(",")) {
- fieldsToUnnest.add(field);
- }
+ return Arrays.asList(_fieldsToUnnest.split("\\s*,\\s*"));
+ } else {
+ return Collections.emptyList();
}
- return fieldsToUnnest;
}
- private ComplexTypeConfig.CollectionNotUnnestedToJson
getcollectionNotUnnestedToJson() {
+ private ComplexTypeConfig.CollectionNotUnnestedToJson
getCollectionNotUnnestedToJson() {
if (_collectionNotUnnestedToJson == null) {
return ComplexTypeTransformer.DEFAULT_COLLECTION_TO_JSON_MODE;
}
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 38d2be7..ce0a9b2 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
@@ -20,7 +20,8 @@ package org.apache.pinot.tools.admin.command;
import java.io.File;
import java.io.FileWriter;
-import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -59,8 +60,8 @@ public class JsonToPinotSchema extends
AbstractBaseAdminCommand implements Comma
@Option(name = "-metrics", metaVar = "<string>", usage = "Comma separated
metric column names.")
String _metrics;
- @Option(name = "-dateTimeColumnName", metaVar = "<string>", usage = "Name of
the dateTime column.")
- String _dateTimeColumnName;
+ @Option(name = "-timeColumnName", metaVar = "<string>", usage = "Name of the
dateTime column.")
+ String _timeColumnName;
@Option(name = "-timeUnit", metaVar = "<string>", usage = "Unit of the time
column (default DAYS).")
TimeUnit _timeUnit = TimeUnit.DAYS;
@@ -68,12 +69,12 @@ public class JsonToPinotSchema extends
AbstractBaseAdminCommand implements Comma
@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")
+ @Option(name = "-delimiter", metaVar = "<string>", usage = "The delimiter
separating components in nested "
+ + "structure, default to dot")
String _delimiter;
- @Option(name = "-collectionNotUnnestedToJson", metaVar = "<string>",
- usage = "The mode of converting collection to JSON string, can be
NONE/NON_PRIMITIVE/ALL")
+ @Option(name = "-collectionNotUnnestedToJson", metaVar = "<string>", usage =
"The mode of converting collection to "
+ + "JSON string, can be NONE/NON_PRIMITIVE/ALL")
String _collectionNotUnnestedToJson;
@SuppressWarnings("FieldCanBeLocal")
@@ -83,16 +84,15 @@ public class JsonToPinotSchema extends
AbstractBaseAdminCommand implements Comma
@Override
public boolean execute()
throws Exception {
- if (_dimensions == null && _metrics == null && _dateTimeColumnName ==
null) {
+ if (_dimensions == null && _metrics == null && _timeColumnName == null) {
LOGGER.error(
"Error: Missing required argument, please specify at least one of
-dimensions, -metrics, -timeColumnName");
return false;
}
Schema schema;
- schema = JsonUtils
- .getPinotSchemaFromJsonFile(new File(_jsonFile), buildFieldTypesMap(),
_timeUnit, buildfieldsToUnnest(),
- getDelimiter(), getcollectionNotUnnestedToJson());
+ schema = JsonUtils.getPinotSchemaFromJsonFile(new File(_jsonFile),
buildFieldTypesMap(), _timeUnit,
+ buildFieldsToUnnest(), getDelimiter(),
getCollectionNotUnnestedToJson());
schema.setSchemaName(_pinotSchemaName);
File outputDir = new File(_outputDir);
@@ -124,7 +124,7 @@ 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 + " _fieldsToUnnest
" + _fieldsToUnnest + " _delimiter "
+ + _timeColumnName + " -timeUnit " + _timeUnit + " _fieldsToUnnest " +
_fieldsToUnnest + " _delimiter "
+ _delimiter + " _collectionNotUnnestedToJson " +
_collectionNotUnnestedToJson;
}
@@ -146,23 +146,21 @@ public class JsonToPinotSchema extends
AbstractBaseAdminCommand implements Comma
fieldTypes.put(column, FieldSpec.FieldType.METRIC);
}
}
- if (_dateTimeColumnName != null) {
- fieldTypes.put(_dateTimeColumnName, FieldSpec.FieldType.DATE_TIME);
+ if (_timeColumnName != null) {
+ fieldTypes.put(_timeColumnName, FieldSpec.FieldType.DATE_TIME);
}
return fieldTypes;
}
- private List<String> buildfieldsToUnnest() {
- List<String> fieldsToUnnest = new ArrayList<>();
+ private List<String> buildFieldsToUnnest() {
if (_fieldsToUnnest != null) {
- for (String field : _fieldsToUnnest.split(",")) {
- fieldsToUnnest.add(field.trim());
- }
+ return Arrays.asList(_fieldsToUnnest.split("\\s*,\\s*"));
+ } else {
+ return Collections.emptyList();
}
- return fieldsToUnnest;
}
- private ComplexTypeConfig.CollectionNotUnnestedToJson
getcollectionNotUnnestedToJson() {
+ private ComplexTypeConfig.CollectionNotUnnestedToJson
getCollectionNotUnnestedToJson() {
if (_collectionNotUnnestedToJson == null) {
return ComplexTypeTransformer.DEFAULT_COLLECTION_TO_JSON_MODE;
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]