From 4db22839ba14719afadd4f371db4bcf2797e9c75 Mon Sep 17 00:00:00 2001 From: zhaojiandong Date: Fri, 18 Jan 2019 15:49:48 +0800 Subject: [PATCH 1/6] Squashed commit of the following: commit f4d29ce2be6d83f77b6fbc4039276a57825cc4bb Author: zhaojiandong Date: Fri Jan 18 15:37:32 2019 +0800 allow user to specify time at ingestion time commit 28a060bf5fd0b4e33fbcca57cb819cd416490802 Author: zhaojiandong Date: Fri Jan 18 13:33:11 2019 +0800 allow user to specify time at ingestion time --- .../benchmark/FlattenJSONBenchmarkUtil.java | 10 ++-- .../druid/benchmark/TimeParseBenchmark.java | 2 +- codestyle/joda-time-forbidden-apis.txt | 2 +- .../data/input/impl/TimeAndDimsParseSpec.java | 2 +- .../druid/data/input/impl/TimestampSpec.java | 13 ++++- .../druid/java/util/common/DateTimes.java | 57 ++++++++++++++----- .../util/common/parsers/TimestampParser.java | 13 +++-- .../org/apache/druid/math/expr/Function.java | 2 +- .../data/input/impl/CSVParseSpecTest.java | 2 + .../input/impl/DelimitedParseSpecTest.java | 5 +- .../input/impl/FileIteratingFirehoseTest.java | 2 +- .../input/impl/InputRowParserSerdeTest.java | 10 ++-- .../impl/JSONLowercaseParseSpecTest.java | 1 + .../data/input/impl/JSONParseSpecTest.java | 6 +- .../input/impl/JavaScriptParseSpecTest.java | 6 +- .../druid/data/input/impl/ParseSpecTest.java | 4 ++ .../data/input/impl/RegexParseSpecTest.java | 2 +- .../data/input/impl/SqlFirehoseTest.java | 2 +- .../input/impl/StringInputRowParserTest.java | 4 +- .../input/impl/TimeAndDimsParseSpecTest.java | 2 +- .../data/input/impl/TimestampSpecTest.java | 34 +++++++++-- ...fetchableTextFilesFirehoseFactoryTest.java | 1 + .../common/parsers/TimestampParserTest.java | 26 ++++----- .../data/input/influx/InfluxParseSpec.java | 2 +- .../orc/OrcHadoopInputRowParserTest.java | 4 +- .../input/orc/OrcIndexGeneratorJobTest.java | 2 +- .../thrift/ThriftInputRowParserTest.java | 4 +- .../TimestampAggregatorFactory.java | 2 +- .../segment/MapVirtualColumnSelectTest.java | 2 +- .../segment/MapVirtualColumnTestBase.java | 2 +- .../druid/data/input/avro/AvroParseSpec.java | 2 +- .../input/AvroStreamInputRowParserTest.java | 4 +- .../query/filter/BloomDimFilterTest.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../kinesis/KinesisIndexTaskTest.java | 2 +- .../supervisor/KinesisSupervisorTest.java | 2 +- .../protobuf/ProtobufInputRowParserTest.java | 4 +- .../indexer/BatchDeltaIngestionTest.java | 2 +- .../DetermineHashedPartitionsJobTest.java | 2 +- .../indexer/DeterminePartitionsJobTest.java | 2 +- .../indexer/HadoopDruidIndexerMapperTest.java | 2 +- .../indexer/IndexGeneratorCombinerTest.java | 2 +- .../druid/indexer/IndexGeneratorJobTest.java | 12 ++-- .../apache/druid/indexer/JobHelperTest.java | 2 +- .../indexer/path/DatasourcePathSpecTest.java | 2 +- .../updater/HadoopConverterJobTest.java | 2 +- ...penderatorDriverRealtimeIndexTaskTest.java | 2 +- .../common/task/CompactionTaskRunTest.java | 1 + .../indexing/common/task/IndexTaskTest.java | 10 ++++ .../common/task/RealtimeIndexTaskTest.java | 2 +- ...stractParallelIndexSupervisorTaskTest.java | 1 + .../ParallelIndexSupervisorTaskSerdeTest.java | 1 + .../IngestSegmentFirehoseFactoryTest.java | 4 +- ...estSegmentFirehoseFactoryTimelineTest.java | 2 +- .../expression/TimestampParseExprMacro.java | 4 +- .../query/extraction/TimeDimExtractionFn.java | 4 +- .../incremental/IncrementalIndexSchema.java | 2 +- .../apache/druid/query/DoubleStorageTest.java | 2 +- .../druid/query/MultiValuedDimensionTest.java | 2 +- .../druid/query/SchemaEvolutionTest.java | 2 +- .../GroupByQueryRunnerFactoryTest.java | 2 +- .../metadata/SegmentMetadataQueryTest.java | 2 +- .../apache/druid/segment/MetadataTest.java | 10 ++-- .../org/apache/druid/segment/TestIndex.java | 4 +- .../druid/segment/filter/AndFilterTest.java | 2 +- .../druid/segment/filter/BoundFilterTest.java | 2 +- .../filter/ColumnComparisonFilterTest.java | 2 +- .../segment/filter/ExpressionFilterTest.java | 2 +- .../segment/filter/FilterPartitionTest.java | 2 +- .../filter/FloatAndDoubleFilteringTest.java | 2 +- .../druid/segment/filter/InFilterTest.java | 2 +- .../segment/filter/InvalidFilteringTest.java | 2 +- .../segment/filter/JavaScriptFilterTest.java | 2 +- .../druid/segment/filter/LikeFilterTest.java | 2 +- .../segment/filter/LongFilteringTest.java | 2 +- .../druid/segment/filter/NotFilterTest.java | 2 +- .../druid/segment/filter/RegexFilterTest.java | 2 +- .../segment/filter/SearchQueryFilterTest.java | 2 +- .../segment/filter/SelectorFilterTest.java | 2 +- .../segment/filter/TimeFilteringTest.java | 2 +- .../IncrementalIndexMultiValueSpecTest.java | 2 +- .../segment/indexing/DataSchemaTest.java | 14 ++--- .../segment/indexing/TransformSpecTest.java | 2 +- .../segment/realtime/FireDepartmentTest.java | 1 + .../appenderator/AppenderatorTester.java | 2 +- ...DefaultOfflineAppenderatorFactoryTest.java | 2 +- .../EventReceiverFirehoseIdleTest.java | 1 + .../firehose/EventReceiverFirehoseTest.java | 2 + .../firehose/IngestSegmentFirehoseTest.java | 2 +- .../firehose/LocalFirehoseFactoryTest.java | 1 + .../firehose/SqlFirehoseFactoryTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 4 +- .../druid/sql/calcite/planner/Calcites.java | 4 +- .../druid/sql/calcite/util/CalciteTests.java | 2 +- 95 files changed, 250 insertions(+), 154 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java index 6c0ec31d64a6..d1612982b345 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java @@ -54,7 +54,7 @@ public FlattenJSONBenchmarkUtil() public Parser getFlatParser() { JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(null, null, null), null, null @@ -68,7 +68,7 @@ public Parser getFieldDiscoveryParser() JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(null, null, null), flattenSpec, null @@ -111,7 +111,7 @@ public Parser getNestedParser() JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(null, null, null), flattenSpec, null @@ -154,7 +154,7 @@ public Parser getForcedPathParser() JSONPathSpec flattenSpec = new JSONPathSpec(false, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(null, null, null), flattenSpec, null @@ -195,7 +195,7 @@ public Parser getJqParser() JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(null, null, null), flattenSpec, null diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java index 474cc79df913..647854b35de0 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/TimeParseBenchmark.java @@ -53,7 +53,7 @@ public class TimeParseBenchmark static final String DATA_FORMAT = "MM/dd/yyyy HH:mm:ss Z"; - static Function timeFn = TimestampParser.createTimestampParser(DATA_FORMAT); + static Function timeFn = TimestampParser.createTimestampParser(DATA_FORMAT, null); private String[] rows; diff --git a/codestyle/joda-time-forbidden-apis.txt b/codestyle/joda-time-forbidden-apis.txt index f2115b7abc8a..0933f796f979 100644 --- a/codestyle/joda-time-forbidden-apis.txt +++ b/codestyle/joda-time-forbidden-apis.txt @@ -60,7 +60,7 @@ org.joda.time.MutableInterval#parse(java.lang.String) org.joda.time.DateTimeZone#forID(java.lang.String) @ Use DateTimes.inferTzFromString() instead -@defaultMessage Uses default time zone, use DateTimes.UtcFormatter to parse. +@defaultMessage Uses default time zone, use DateTimes.Formatter or DateTimes.UtcFormatter to parse. org.joda.time.format.DateTimeFormatter#parseInto(org.joda.time.ReadWritableInstant, java.lang.String, int) org.joda.time.format.DateTimeFormatter#parseDateTime(java.lang.String) org.joda.time.format.DateTimeFormatter#parseMutableDateTime(java.lang.String) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpec.java index 969ea9a908e3..d6083d70b0ba 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpec.java @@ -37,7 +37,7 @@ public TimeAndDimsParseSpec( ) { super( - timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null), + timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null, null), dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null) ); } diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java index a61f98370f2c..ea7ef852a0df 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java @@ -22,7 +22,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Function; +import org.apache.commons.lang3.StringUtils; import org.apache.druid.guice.annotations.PublicApi; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.parsers.TimestampParser; import org.joda.time.DateTime; @@ -49,7 +51,10 @@ private static class ParseCtx private final String timestampFormat; // this value should never be set for production data private final DateTime missingValue; - /** This field is a derivative of {@link #timestampFormat}; not checked in {@link #equals} and {@link #hashCode} */ + private final String timeZone; + /** + * This field is a derivative of {@link #timestampFormat}; not checked in {@link #equals} and {@link #hashCode} + */ private final Function timestampConverter; // remember last value parsed @@ -60,12 +65,14 @@ public TimestampSpec( @JsonProperty("column") String timestampColumn, @JsonProperty("format") String format, // this value should never be set for production data - @JsonProperty("missingValue") DateTime missingValue + @JsonProperty("missingValue") DateTime missingValue, + @JsonProperty("timeZone") String timeZone ) { this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn; this.timestampFormat = format == null ? DEFAULT_FORMAT : format; - this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat); + this.timeZone = (StringUtils.isBlank(timeZone)) ? DateTimes.UTC_TIMEZONE : timeZone; + this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat, this.timeZone); this.missingValue = missingValue == null ? DEFAULT_MISSING_VALUE : missingValue; diff --git a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java index 94f1295d2e69..1c70b17c2f07 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java +++ b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java @@ -20,10 +20,12 @@ package org.apache.druid.java.util.common; import io.netty.util.SuppressForbidden; +import org.apache.commons.lang3.StringUtils; import org.joda.time.Chronology; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.chrono.ISOChronology; +import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.ISODateTimeFormat; @@ -35,10 +37,11 @@ public final class DateTimes public static final DateTime EPOCH = utc(0); public static final DateTime MAX = utc(JodaUtils.MAX_INSTANT); public static final DateTime MIN = utc(JodaUtils.MIN_INSTANT); + public static final String UTC_TIMEZONE = DateTimeZone.UTC.getID(); - public static final UtcFormatter ISO_DATE_TIME = wrapFormatter(ISODateTimeFormat.dateTime()); - public static final UtcFormatter ISO_DATE_OPTIONAL_TIME = wrapFormatter(ISODateTimeFormat.dateOptionalTimeParser()); - public static final UtcFormatter ISO_DATE_OR_TIME_WITH_OFFSET = wrapFormatter( + public static final UtcFormatter ISO_DATE_TIME = wrapUtcFormatter(ISODateTimeFormat.dateTime()); + public static final UtcFormatter ISO_DATE_OPTIONAL_TIME = wrapUtcFormatter(ISODateTimeFormat.dateOptionalTimeParser()); + public static final UtcFormatter ISO_DATE_OR_TIME_WITH_OFFSET = wrapUtcFormatter( ISODateTimeFormat.dateTimeParser().withOffsetParsed() ); @@ -64,18 +67,19 @@ public static DateTimeZone inferTzFromString(String tzId) } } - /** - * Simple wrapper class to enforce UTC Chronology in formatter. Specifically, it will use - * {@link DateTimeFormatter#withChronology(Chronology)} to set the chronology to - * {@link ISOChronology#getInstanceUTC()} on the wrapped {@link DateTimeFormatter}. - */ - public static class UtcFormatter + public static class Formatter { - private final DateTimeFormatter innerFormatter; + private DateTimeFormatter innerFormatter; - private UtcFormatter(final DateTimeFormatter innerFormatter) + public Formatter(DateTimeFormatter innerFormatter) + { + this.innerFormatter = innerFormatter; + } + + public Formatter(String format, String timezone) { - this.innerFormatter = innerFormatter.withChronology(ISOChronology.getInstanceUTC()); + this.innerFormatter = DateTimeFormat.forPattern(format) + .withChronology(ISOChronology.getInstance(inferTzfromString(timezone))); } @SuppressForbidden(reason = "DateTimeFormatter#parseDateTime") @@ -90,12 +94,39 @@ public String print(final DateTime instant) } } + /** + * Simple wrapper class to enforce UTC Chronology in formatter. Specifically, it will use + * {@link DateTimeFormatter#withChronology(Chronology)} to set the chronology to + * {@link ISOChronology#getInstanceUTC()} on the wrapped {@link DateTimeFormatter}. + */ + public static class UtcFormatter extends Formatter + { + private UtcFormatter(final DateTimeFormatter innerFormatter) + { + super(innerFormatter.withChronology(ISOChronology.getInstanceUTC())); + } + } + + /** + * Create a {@link Formatter} by args + * + * @param format + * @param timeZone + * + * @return + */ + public static Formatter wrapFormatter(String format, String timeZone) + { + timeZone = StringUtils.isNotBlank(timeZone) ? timeZone : DateTimeZone.UTC.getID(); + return new Formatter(format, timeZone); + } + /** * Creates a {@link UtcFormatter} that wraps around a {@link DateTimeFormatter}. * * @param formatter inner {@link DateTimeFormatter} used to parse {@link String} */ - public static UtcFormatter wrapFormatter(final DateTimeFormatter formatter) + public static UtcFormatter wrapUtcFormatter(final DateTimeFormatter formatter) { return new UtcFormatter(formatter); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/TimestampParser.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/TimestampParser.java index ecd06f822f93..e819fea04a12 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/TimestampParser.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/TimestampParser.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.IAE; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; import org.joda.time.format.DateTimeFormatterBuilder; import org.joda.time.format.DateTimeParser; @@ -37,12 +36,13 @@ public class TimestampParser { public static Function createTimestampParser( - final String format + final String format, + final String userTimeZone ) { if ("auto".equalsIgnoreCase(format)) { // Could be iso or millis - final DateTimes.UtcFormatter parser = DateTimes.wrapFormatter(createAutoParser()); + final DateTimes.UtcFormatter parser = DateTimes.wrapUtcFormatter(createAutoParser()); return (String input) -> { Preconditions.checkArgument(!Strings.isNullOrEmpty(input), "null timestamp"); @@ -87,7 +87,7 @@ public static Function createTimestampParser( }; } else { try { - final DateTimes.UtcFormatter formatter = DateTimes.wrapFormatter(DateTimeFormat.forPattern(format)); + final DateTimes.Formatter formatter = DateTimes.wrapFormatter(format, userTimeZone); return input -> { Preconditions.checkArgument(!Strings.isNullOrEmpty(input), "null timestamp"); return formatter.parse(ParserUtils.stripQuotes(input)); @@ -117,10 +117,11 @@ public static Function createNumericTimestampParser( } public static Function createObjectTimestampParser( - final String format + final String format, + final String timeZone ) { - final Function stringFun = createTimestampParser(format); + final Function stringFun = createTimestampParser(format, timeZone); final Function numericFun = createNumericTimestampParser(format); return o -> { diff --git a/core/src/main/java/org/apache/druid/math/expr/Function.java b/core/src/main/java/org/apache/druid/math/expr/Function.java index 24247558c133..d6559ff9f770 100644 --- a/core/src/main/java/org/apache/druid/math/expr/Function.java +++ b/core/src/main/java/org/apache/druid/math/expr/Function.java @@ -835,7 +835,7 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) if (format.type() != ExprType.STRING) { throw new IAE("second argument should be string type but got %s type", format.type()); } - formatter = DateTimes.wrapFormatter(DateTimeFormat.forPattern(format.asString())); + formatter = DateTimes.wrapUtcFormatter(DateTimeFormat.forPattern(format.asString())); } DateTime date; try { diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java index f5fa4ee1b9bd..50d951c274cc 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java @@ -35,6 +35,7 @@ public void testColumnMissing() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -57,6 +58,7 @@ public void testComma() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java index 9b0786228d66..b7f5332d9646 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java @@ -37,7 +37,7 @@ public class DelimitedParseSpecTest public void testSerde() throws IOException { DelimitedParseSpec spec = new DelimitedParseSpec( - new TimestampSpec("abc", "iso", null), + new TimestampSpec("abc", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "\u0001", "\u0002", @@ -66,6 +66,7 @@ public void testColumnMissing() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -89,6 +90,7 @@ public void testComma() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -112,6 +114,7 @@ public void testDefaultColumnList() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java b/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java index 7683258f3028..7079411ad56d 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java @@ -77,7 +77,7 @@ public FileIteratingFirehoseTest(List texts, int numSkipHeaderRows) { parser = new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("ts", "auto", null), + new TimestampSpec("ts", "auto", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null), ",", ImmutableList.of("ts", "x"), diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java index e79953ce4a6a..db906ed68a54 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java @@ -49,7 +49,7 @@ public void testStringInputRowParserSerde() throws Exception { final StringInputRowParser parser = new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), null, null), null, null @@ -91,7 +91,7 @@ public void testMapInputRowParserSerde() throws Exception { final MapInputRowParser parser = new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("timeposix", "posix", null), + new TimestampSpec("timeposix", "posix", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), ImmutableList.of("baz"), @@ -124,7 +124,7 @@ public void testMapInputRowParserNumbersSerde() throws Exception { final MapInputRowParser parser = new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("timemillis", "millis", null), + new TimestampSpec("timemillis", "millis", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "values")), ImmutableList.of("toobig", "value"), @@ -165,7 +165,7 @@ private InputRow testCharsetParseHelper(Charset charset) throws Exception { final StringInputRowParser parser = new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), null, null), null, null @@ -205,7 +205,7 @@ public void testFlattenParse() throws Exception JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); final StringInputRowParser parser = new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(null, null, null), flattenSpec, null diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java index 72c320f4939b..7c4d90fd68f6 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java @@ -36,6 +36,7 @@ public void testLowercasing() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java index 4dd13b601a69..d1f0f6db6344 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java @@ -44,7 +44,7 @@ public class JSONParseSpecTest public void testParseRow() { final JSONParseSpec parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), null, null), new JSONPathSpec( true, @@ -85,7 +85,7 @@ public void testParseRow() public void testParseRowWithConditional() { final JSONParseSpec parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo")), null, null), new JSONPathSpec( true, @@ -116,7 +116,7 @@ public void testSerde() throws IOException HashMap feature = new HashMap(); feature.put("ALLOW_UNQUOTED_CONTROL_CHARS", true); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), null, null), null, feature diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java index ac6ecbad9e48..7bbb4f163330 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java @@ -53,7 +53,7 @@ public void testSerde() throws IOException ) ); JavaScriptParseSpec spec = new JavaScriptParseSpec( - new TimestampSpec("abc", "iso", null), + new TimestampSpec("abc", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "abc", JavaScriptConfig.getEnabledInstance() @@ -74,7 +74,7 @@ public void testMakeParser() { final JavaScriptConfig config = JavaScriptConfig.getEnabledInstance(); JavaScriptParseSpec spec = new JavaScriptParseSpec( - new TimestampSpec("abc", "iso", null), + new TimestampSpec("abc", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "function(str) { var parts = str.split(\"-\"); return { one: parts[0], two: parts[1] } }", config @@ -90,7 +90,7 @@ public void testMakeParserNotAllowed() { final JavaScriptConfig config = new JavaScriptConfig(false); JavaScriptParseSpec spec = new JavaScriptParseSpec( - new TimestampSpec("abc", "iso", null), + new TimestampSpec("abc", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "abc", config diff --git a/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java index 1c2e7808e174..e40f7e9a71bc 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java @@ -67,6 +67,7 @@ public void testDuplicateNames() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -90,6 +91,7 @@ public void testDimAndDimExcluOverlap() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -113,6 +115,7 @@ public void testDimExclusionDuplicate() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -159,6 +162,7 @@ public void testDimensionSpecRequired() new TimestampSpec( "timestamp", "auto", + null, null ), null, diff --git a/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java index c31c6f9dbe3d..2840945f9dbb 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java @@ -37,7 +37,7 @@ public class RegexParseSpecTest public void testSerde() throws IOException { RegexParseSpec spec = new RegexParseSpec( - new TimestampSpec("abc", "iso", null), + new TimestampSpec("abc", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "\u0001", Collections.singletonList("abc"), diff --git a/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java b/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java index f1c3d62481ae..05a5c232b8f5 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java @@ -81,7 +81,7 @@ public void setup() throws IOException this.fileList = testFile; parser = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null) ) ); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java b/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java index 332e76f101c4..d26266a305c4 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java @@ -35,7 +35,7 @@ public class StringInputRowParserTest public void testDisableJavaScript() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( @@ -61,7 +61,7 @@ public void testDisableJavaScript() public void testDisableJavaScript2() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java index 656a87d76706..3d5267ffa6b0 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java @@ -63,7 +63,7 @@ public void testSerdeWithNonNulls() throws Exception Assert.assertEquals( new TimeAndDimsParseSpec( - new TimestampSpec("tcol", null, null), + new TimestampSpec("tcol", null, null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null) ), actual diff --git a/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java index ffbca5c1abda..f5374509dd97 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java @@ -31,7 +31,7 @@ public class TimestampSpecTest @Test public void testExtractTimestamp() { - TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null); + TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null, null); Assert.assertEquals( DateTimes.of("2014-03-01"), spec.extractTimestamp(ImmutableMap.of("TIMEstamp", "2014-03-01")) @@ -41,7 +41,7 @@ public void testExtractTimestamp() @Test public void testExtractTimestampWithMissingTimestampColumn() { - TimestampSpec spec = new TimestampSpec(null, null, DateTimes.EPOCH); + TimestampSpec spec = new TimestampSpec(null, null, DateTimes.EPOCH, null); Assert.assertEquals( DateTimes.of("1970-01-01"), spec.extractTimestamp(ImmutableMap.of("dim", "foo")) @@ -59,9 +59,9 @@ public void testContextualTimestampList() "2000-01-01T05:00:02", "2000-01-01T05:00:03", }; - TimestampSpec spec = new TimestampSpec("TIMEstamp", DATE_FORMAT, null); + TimestampSpec spec = new TimestampSpec("TIMEstamp", DATE_FORMAT, null, null); - DateTimes.UtcFormatter formatter = DateTimes.wrapFormatter(ISODateTimeFormat.dateHourMinuteSecond()); + DateTimes.UtcFormatter formatter = DateTimes.wrapUtcFormatter(ISODateTimeFormat.dateHourMinuteSecond()); for (String date : dates) { DateTime dateTime = spec.extractTimestamp(ImmutableMap.of("TIMEstamp", date)); @@ -69,4 +69,30 @@ public void testContextualTimestampList() Assert.assertEquals(expectedDateTime, dateTime); } } + + @Test + public void testExtractTimestampWithTimezone() + { + String DATE_FORMAT = "yyyy-MM-dd HH:mm:ss"; + String timeZone = "Asia/Shanghai"; + String[] dates = new String[]{ + "2000-01-01 00:00:00", + "2000-01-01 08:00:00", + "2000-01-01 12:00:01", + "2000-01-01 23:59:59", + }; + TimestampSpec spec = new TimestampSpec("log_time", DATE_FORMAT, null, timeZone); + + DateTimes.Formatter expectedFormatter = DateTimes.wrapFormatter(DATE_FORMAT, timeZone); + + for (String date : dates) { + DateTime actualDateTime = spec.extractTimestamp(ImmutableMap.of( + "log_time", date, + "dim1", "value1" + )); + long expectedTs = expectedFormatter.parse(date).getMillis(); + Assert.assertEquals(expectedTs, actualDateTime.getMillis()); + } + } + } diff --git a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java index c1d1b318d00d..88b44db775da 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java @@ -69,6 +69,7 @@ public class PrefetchableTextFilesFirehoseFactoryTest new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( diff --git a/core/src/test/java/org/apache/druid/java/util/common/parsers/TimestampParserTest.java b/core/src/test/java/org/apache/druid/java/util/common/parsers/TimestampParserTest.java index 9c0e66e10d22..c53e57ca96ec 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/parsers/TimestampParserTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/parsers/TimestampParserTest.java @@ -58,7 +58,7 @@ public void testExtractTimeZone() @Test public void testAuto() { - final Function parser = TimestampParser.createObjectTimestampParser("auto"); + final Function parser = TimestampParser.createObjectTimestampParser("auto", null); Assert.assertEquals(DateTimes.of("2009-02-13T23:31:30Z"), parser.apply("1234567890000")); Assert.assertEquals(DateTimes.of("2009-02-13T23:31:30Z"), parser.apply("2009-02-13T23:31:30Z")); Assert.assertEquals(DateTimes.of("2009-02-13T23:31:30-08:00"), parser.apply("2009-02-13T23:31:30-08:00")); @@ -78,7 +78,7 @@ public void testAuto() @Test public void testAutoNull() { - final Function parser = TimestampParser.createObjectTimestampParser("auto"); + final Function parser = TimestampParser.createObjectTimestampParser("auto", null); expectedException.expect(NullPointerException.class); parser.apply(null); @@ -87,7 +87,7 @@ public void testAutoNull() @Test public void testAutoInvalid() { - final Function parser = TimestampParser.createObjectTimestampParser("auto"); + final Function parser = TimestampParser.createObjectTimestampParser("auto", null); expectedException.expect(IllegalArgumentException.class); parser.apply("asdf"); @@ -96,7 +96,7 @@ public void testAutoInvalid() @Test public void testRuby() { - final Function parser = TimestampParser.createObjectTimestampParser("ruby"); + final Function parser = TimestampParser.createObjectTimestampParser("ruby", null); Assert.assertEquals(DateTimes.of("2013-01-16T14:41:47.435Z"), parser.apply("1358347307.435447")); Assert.assertEquals(DateTimes.of("2013-01-16T14:41:47.435Z"), parser.apply(1358347307.435447D)); } @@ -106,7 +106,7 @@ public void testNano() { String timeNsStr = "1427504794977098494"; DateTime expectedDt = DateTimes.of("2015-3-28T01:06:34.977Z"); - final Function parser = TimestampParser.createObjectTimestampParser("nano"); + final Function parser = TimestampParser.createObjectTimestampParser("nano", null); Assert.assertEquals("Incorrect truncation of nanoseconds -> milliseconds", expectedDt, parser.apply(timeNsStr)); @@ -122,7 +122,7 @@ public void testNano() public void testTimeStampParserWithQuotes() { DateTime d = new DateTime(1994, 11, 9, 4, 0, DateTimeZone.forOffsetHours(-8)); - Function parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss z yyyy"); + Function parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss z yyyy", null); Assert.assertEquals(d.getMillis(), parser.apply(" \" Wed Nov 9 04:00:00 PST 1994 \" ").getMillis()); } @@ -130,7 +130,7 @@ public void testTimeStampParserWithQuotes() public void testTimeStampParserWithShortTimeZone() { DateTime d = new DateTime(1994, 11, 9, 4, 0, DateTimeZone.forOffsetHours(-8)); - Function parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss z yyyy"); + Function parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss z yyyy", null); Assert.assertEquals(d.getMillis(), parser.apply("Wed Nov 9 04:00:00 PST 1994").getMillis()); } @@ -141,13 +141,13 @@ public void testTimeStampParserWithLongTimeZone() long millis1 = new DateTime(1994, 11, 9, 4, 0, DateTimeZone.forOffsetHours(-8)).getMillis(); long millis2 = new DateTime(1994, 11, 9, 4, 0, DateTimeZone.forOffsetHours(-6)).getMillis(); - Function parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss zZ z yyyy"); + Function parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss zZ z yyyy", null); Assert.assertEquals(millis1, parser.apply("Wed Nov 9 04:00:00 GMT-0800 PST 1994").getMillis()); Assert.assertEquals(millis2, parser.apply("Wed Nov 9 04:00:00 GMT-0600 CST 1994").getMillis()); Assert.assertEquals(millis1, parser.apply("Wed Nov 9 04:00:00 UTC-0800 PST 1994").getMillis()); Assert.assertEquals(millis2, parser.apply("Wed Nov 9 04:00:00 UTC-0600 CST 1994").getMillis()); - parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss zZ yyyy"); + parser = TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss zZ yyyy", null); Assert.assertEquals(millis1, parser.apply("Wed Nov 9 04:00:00 GMT-0800 1994").getMillis()); Assert.assertEquals(millis2, parser.apply("Wed Nov 9 04:00:00 GMT-0600 1994").getMillis()); Assert.assertEquals(millis1, parser.apply("Wed Nov 9 04:00:00 UTC-0800 1994").getMillis()); @@ -157,11 +157,11 @@ public void testTimeStampParserWithLongTimeZone() @Test public void testTimeZoneAtExtremeLocations() { - Function parser = TimestampParser.createTimestampParser("EEE MMM dd yy HH:mm:ss zZ z"); + Function parser = TimestampParser.createTimestampParser("EEE MMM dd yy HH:mm:ss zZ z", null); Assert.assertEquals(new DateTime(2005, 1, 22, 13, 0, DateTimeZone.forOffsetHours(-6)).getMillis(), parser.apply("Sat Jan 22 05 13:00:00 GMT-0600 CST").getMillis()); - parser = TimestampParser.createTimestampParser("zZ z EEE MMM dd yy HH:mm:ss"); + parser = TimestampParser.createTimestampParser("zZ z EEE MMM dd yy HH:mm:ss", null); Assert.assertEquals(new DateTime(2005, 1, 22, 13, 0, DateTimeZone.forOffsetHours(-6)).getMillis(), parser.apply("GMT-0600 CST Sat Jan 22 05 13:00:00").getMillis()); } @@ -171,12 +171,12 @@ public void testJodaSymbolInsideLiteral() { DateTime d = new DateTime(1994, 11, 9, 4, 0, DateTimeZone.forOffsetHours(-8)); Assert.assertEquals(d.getMillis(), - TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss z yyyy 'helloz'") + TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss z yyyy 'helloz'", null) .apply("Wed Nov 9 04:00:00 PST 1994 helloz") .getMillis() ); Assert.assertEquals(d.getMillis(), - TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss 'helloz' z yyyy 'hello'") + TimestampParser.createTimestampParser("EEE MMM dd HH:mm:ss 'helloz' z yyyy 'hello'", null) .apply("Wed Nov 9 04:00:00 helloz PST 1994 hello") .getMillis() ); diff --git a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java index 025c3ad08e54..209ee2011475 100644 --- a/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java +++ b/extensions-contrib/influx-extensions/src/main/java/org/apache/druid/data/input/influx/InfluxParseSpec.java @@ -40,7 +40,7 @@ public InfluxParseSpec( ) { super( - new TimestampSpec(InfluxParser.TIMESTAMP_KEY, "millis", null), + new TimestampSpec(InfluxParser.TIMESTAMP_KEY, "millis", null, null), dimensionsSpec != null ? dimensionsSpec : new DimensionsSpec(null, null, null) ); this.measurementWhitelist = measurementWhitelist; diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java index 3b7f56928bb9..4fcc546cd8c5 100644 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java @@ -108,6 +108,7 @@ public void testSerde() throws IOException new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -130,6 +131,7 @@ public void testTypeFromParseSpec() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( @@ -150,7 +152,7 @@ public void testParse() final String typeString = "struct,col3:float,col4:bigint,col5:decimal,col6:array,col7:map>"; final OrcHadoopInputRowParser parser = new OrcHadoopInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec(null, null, null) ), typeString, diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java index fd9b9b6ee467..6798bcbc2abd 100644 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -132,7 +132,7 @@ public class OrcIndexGeneratorJobTest }; private final InputRowParser inputRowParser = new OrcHadoopInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null) ), "struct", diff --git a/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java b/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java index 3d890a7c9ffb..eebf87a61677 100644 --- a/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java +++ b/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java @@ -59,7 +59,7 @@ public class ThriftInputRowParserTest @Before public void setUp() { - parseSpec = new JSONParseSpec(new TimestampSpec("date", "auto", null), + parseSpec = new JSONParseSpec(new TimestampSpec("date", "auto", null, null), new DimensionsSpec(Lists.newArrayList( new StringDimensionSchema("title"), new StringDimensionSchema("lastName") @@ -121,7 +121,7 @@ public void testParse() throws Exception public void testDisableJavaScript() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( diff --git a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java index bf9b69020d39..168786671270 100644 --- a/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java +++ b/extensions-contrib/time-min-max/src/main/java/org/apache/druid/query/aggregation/TimestampAggregatorFactory.java @@ -58,7 +58,7 @@ public class TimestampAggregatorFactory extends AggregatorFactory this.comparator = comparator; this.initValue = initValue; - this.timestampSpec = new TimestampSpec(fieldName, timeFormat, null); + this.timestampSpec = new TimestampSpec(fieldName, timeFormat, null, null); } @Override diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java index f248bec0d3bb..df5dd767f4c9 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java @@ -87,7 +87,7 @@ public static Iterable constructorFeeder() throws IOException final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("dim", "keys", "values")), null, null), "\t", ",", diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java index ffc3706f95e5..c09f104b91bb 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java @@ -45,7 +45,7 @@ static IncrementalIndex generateIndex() throws IOException final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "auto", null), + new TimestampSpec("ts", "auto", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("dim", "keys", "values")), null, null), "\t", ",", diff --git a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParseSpec.java b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParseSpec.java index c7912e38dde3..034f84b8f4f3 100644 --- a/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParseSpec.java +++ b/extensions-core/avro-extensions/src/main/java/org/apache/druid/data/input/avro/AvroParseSpec.java @@ -38,7 +38,7 @@ public AvroParseSpec( ) { super( - timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null), + timestampSpec != null ? timestampSpec : new TimestampSpec(null, null, null, null), dimensionsSpec != null ? dimensionsSpec : DimensionsSpec.EMPTY, flattenSpec != null ? flattenSpec : JSONPathSpec.DEFAULT ); diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java index 184c8b247023..65a237a9c787 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java @@ -99,7 +99,7 @@ public class AvroStreamInputRowParserTest "timestamp" ); public static final AvroParseSpec PARSE_SPEC = new AvroParseSpec( - new TimestampSpec("nested", "millis", null), + new TimestampSpec("nested", "millis", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS), Collections.emptyList(), null), new JSONPathSpec( true, @@ -109,7 +109,7 @@ public class AvroStreamInputRowParserTest ) ); public static final AvroParseSpec PARSE_SPEC_SCHEMALESS = new AvroParseSpec( - new TimestampSpec("nested", "millis", null), + new TimestampSpec("nested", "millis", null, null), new DimensionsSpec(null, null, null), new JSONPathSpec( true, diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java index fb8d31aa4a78..d8c4e7556193 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java @@ -61,7 +61,7 @@ public class BloomDimFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3", "dim6")), null, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index b7b389668a46..dba2ac48d064 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -235,7 +235,7 @@ public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec( Arrays.asList( new StringDimensionSchema("dim1"), diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index a08806acc173..6bec67b17df2 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -2860,7 +2860,7 @@ private static DataSchema getDataSchema(String dataSource) objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec( dimensions, null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index a8130b9c0e5b..40009b284008 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -254,7 +254,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec( Arrays.asList( new StringDimensionSchema("dim1"), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index 750b9247d792..f698ef9b8d4a 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -3536,7 +3536,7 @@ private static DataSchema getDataSchema(String dataSource) objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec( dimensions, null, diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java index 081fc23a24f6..f0d4939293e1 100644 --- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java +++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java @@ -58,7 +58,7 @@ public class ProtobufInputRowParserTest public void setUp() { parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(Lists.newArrayList( new StringDimensionSchema("event"), new StringDimensionSchema("id"), @@ -181,7 +181,7 @@ public void testParse() throws Exception public void testDisableJavaScript() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index b68272460472..f5253940c590 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -447,7 +447,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "host2", "visited_num"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java index bbec6766419d..4ea346d91ec6 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -146,7 +146,7 @@ public DetermineHashedPartitionsJobTest( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", null, null), + new TimestampSpec("ts", null, null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of( "market", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java index 51f4120752de..078b4ae5de98 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java @@ -224,7 +224,7 @@ public DeterminePartitionsJobTest( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")), null, null), null, ImmutableList.of("timestamp", "host", "country", "visited_num"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java index 43ca77d6217b..f65b652983c1 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -64,7 +64,7 @@ public class HadoopDruidIndexerMapperTest JSON_MAPPER.convertValue( new HadoopyStringInputRowParser( new JSONParseSpec( - new TimestampSpec("t", "auto", null), + new TimestampSpec("t", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")), null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java index aeadd46adb66..4931dc7ab977 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java @@ -69,7 +69,7 @@ public void setUp() throws Exception HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords")), null, null) ), null diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index 8e191c7fba91..9d8443340376 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -142,7 +142,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), @@ -191,7 +191,7 @@ public static Collection constructFeed() null, new HadoopyStringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), @@ -239,7 +239,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), @@ -298,7 +298,7 @@ public static Collection constructFeed() SequenceFileInputFormat.class.getName(), new HadoopyStringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), @@ -332,7 +332,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "yyyyMMddHH", null), + new TimestampSpec("ts", "yyyyMMddHH", null, null), new DimensionsSpec(null, null, null), null, null @@ -365,7 +365,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "yyyyMMddHH", null), + new TimestampSpec("ts", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("B", "F", "M", "Q", "X", "Y")), null, null), null, null diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index 4aa6a5904cd4..a51607cce25b 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -80,7 +80,7 @@ public void setup() throws Exception HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java index a229d8be348f..3fbf0383542e 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java @@ -327,7 +327,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig() HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), new DimensionsSpec(null, null, null), null, ImmutableList.of("timestamp", "host", "visited"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java index f81c02a623be..fbae209654d5 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java @@ -161,7 +161,7 @@ public InputStream openStream() throws IOException HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList(TestIndex.DIMENSIONS)), null, null), "\t", "\u0001", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index b57397ab8cd6..079cf6220f21 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1378,7 +1378,7 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( TestHelper.makeJsonMapper().convertValue( new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", null), + new TimestampSpec("t", "auto", null, null), new DimensionsSpec( ImmutableList.of( new StringDimensionSchema("dim1"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 54796a542fd2..a7c87cd5c4b7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -82,6 +82,7 @@ public class CompactionTaskRunTest extends IngestionTestBase new TimestampSpec( "ts", "auto", + null, null ), new DimensionsSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index ef7dbce7b8d4..0c0ae758537b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -132,6 +132,7 @@ public class IndexTaskTest new TimestampSpec( "ts", "auto", + null, null ), new DimensionsSpec( @@ -667,6 +668,7 @@ public void testCSVFileWithHeader() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -719,6 +721,7 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -928,6 +931,7 @@ public void testIgnoreParseException() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -981,6 +985,7 @@ public void testReportParseException() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -1070,6 +1075,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -1193,6 +1199,7 @@ public void testMultipleParseExceptionsFailure() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -1309,6 +1316,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -1412,6 +1420,7 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( @@ -1482,6 +1491,7 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception new TimestampSpec( "time", "auto", + null, null ), new DimensionsSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 2c3d3c3d8730..861e7eca298c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -893,7 +893,7 @@ private RealtimeIndexTask makeRealtimeTask( TestHelper.makeJsonMapper().convertValue( new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", null), + new TimestampSpec("t", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim1t")), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 5d42919fa49f..3789ace16935 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -82,6 +82,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase new TimestampSpec( "ts", "auto", + null, null ), new DimensionsSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 0fa747f6604f..5ed440a8014c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -63,6 +63,7 @@ public class ParallelIndexSupervisorTaskSerdeTest new TimestampSpec( "ts", "auto", + null, null ), new DimensionsSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 3329419cf337..5b30302f33e0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -326,7 +326,7 @@ public DataSegment restore(DataSegment segment) ROW_PARSER, new MapInputRowParser( new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), + new TimestampSpec(TIME_COLUMN, "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of()), ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME), @@ -441,7 +441,7 @@ public IngestSegmentFirehoseFactoryTest( private static final InputRowParser> ROW_PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), + new TimestampSpec(TIME_COLUMN, "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of(DIM_NAME)), ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 70e877bad45b..35def5ec2bb0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -99,7 +99,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest private static final InputRowParser> ROW_PARSER = TransformSpec.NONE.decorate( new MapInputRowParser( new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), + new TimestampSpec(TIME_COLUMN, "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS)), null, diff --git a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java index 2b65fb54653e..53f202e96b52 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/TimestampParseExprMacro.java @@ -62,7 +62,7 @@ public Expr apply(final List args) final DateTimes.UtcFormatter formatter = formatString == null ? createDefaultParser(timeZone) - : DateTimes.wrapFormatter(DateTimeFormat.forPattern(formatString).withZone(timeZone)); + : DateTimes.wrapUtcFormatter(DateTimeFormat.forPattern(formatString).withZone(timeZone)); class TimestampParseExpr implements Expr { @@ -118,7 +118,7 @@ private static DateTimes.UtcFormatter createDefaultParser(final DateTimeZone tim .appendOptional(offsetElement.getParser()) .toParser(); - return DateTimes.wrapFormatter( + return DateTimes.wrapUtcFormatter( new DateTimeFormatterBuilder() .append(ISODateTimeFormat.dateElementParser()) .appendOptional(timeOrOffset) diff --git a/processing/src/main/java/org/apache/druid/query/extraction/TimeDimExtractionFn.java b/processing/src/main/java/org/apache/druid/query/extraction/TimeDimExtractionFn.java index 8b58a8198593..1962aee7712d 100644 --- a/processing/src/main/java/org/apache/druid/query/extraction/TimeDimExtractionFn.java +++ b/processing/src/main/java/org/apache/druid/query/extraction/TimeDimExtractionFn.java @@ -65,8 +65,8 @@ public TimeDimExtractionFn( private Supplier> makeFunctionSupplier() { if (joda) { - final DateTimes.UtcFormatter parser = DateTimes.wrapFormatter(DateTimeFormat.forPattern(timeFormat)); - final DateTimes.UtcFormatter formatter = DateTimes.wrapFormatter(DateTimeFormat.forPattern(resultFormat)); + final DateTimes.UtcFormatter parser = DateTimes.wrapUtcFormatter(DateTimeFormat.forPattern(timeFormat)); + final DateTimes.UtcFormatter formatter = DateTimes.wrapUtcFormatter(DateTimeFormat.forPattern(resultFormat)); final Function fn = value -> { DateTime date; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java index 87d75f8c7e2e..8671ee7c0960 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java @@ -132,7 +132,7 @@ public Builder withTimestampSpec(InputRowParser parser) && parser.getParseSpec().getTimestampSpec() != null) { this.timestampSpec = parser.getParseSpec().getTimestampSpec(); } else { - this.timestampSpec = new TimestampSpec(null, null, null); + this.timestampSpec = new TimestampSpec(null, null, null, null); } return this; } diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index a85b41d61165..8e0c2b6e2311 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -121,7 +121,7 @@ private Druids.ScanQueryBuilder newTestQuery() private static final InputRowParser> ROW_PARSER = new MapInputRowParser( new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), + new TimestampSpec(TIME_COLUMN, "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of(DIM_NAME)), ImmutableList.of(DIM_FLOAT_NAME), diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index c4c4a0b99af6..4b0770ebf165 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -121,7 +121,7 @@ public void setup() throws Exception StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("product", "tags")), null, null), "\t", ImmutableList.of("timestamp", "product", "tags"), diff --git a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java index c15a2e60deb8..d90eeb0f1840 100644 --- a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java @@ -79,7 +79,7 @@ public static List inputRowsWithDimensions(final List dimensio { final MapInputRowParser parser = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(dimensions), dimensions.isEmpty() ? ImmutableList.of("t", "c1", "c2") : null, diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index 56598e41d6f2..a8fe64263a99 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -149,7 +149,7 @@ private Segment createSegment() throws Exception StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "iso", null), + new TimestampSpec("timestamp", "iso", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("product", "tags")), null, null), "\t", ImmutableList.of("timestamp", "product", "tags"), diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java index 31e70d376644..0f2df13e7408 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java @@ -719,7 +719,7 @@ public void testSegmentMetadataQueryWithTimestampSpecMerge() 0, expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(), null, - new TimestampSpec("ds", "auto", null), + new TimestampSpec("ds", "auto", null, null), null, null ); diff --git a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java index c763e66b1ed4..ec65cf5ab070 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -81,7 +81,7 @@ public void testMerge() final Metadata m1 = new Metadata( Collections.singletonMap("k", "v"), aggs, - new TimestampSpec("ds", "auto", null), + new TimestampSpec("ds", "auto", null, null), Granularities.ALL, Boolean.FALSE ); @@ -89,7 +89,7 @@ public void testMerge() final Metadata m2 = new Metadata( Collections.singletonMap("k", "v"), aggs, - new TimestampSpec("ds", "auto", null), + new TimestampSpec("ds", "auto", null, null), Granularities.ALL, Boolean.FALSE ); @@ -97,7 +97,7 @@ public void testMerge() final Metadata m3 = new Metadata( Collections.singletonMap("k", "v"), aggs, - new TimestampSpec("ds", "auto", null), + new TimestampSpec("ds", "auto", null, null), Granularities.ALL, Boolean.TRUE ); @@ -107,7 +107,7 @@ public void testMerge() new AggregatorFactory[]{ new LongMaxAggregatorFactory("n", "n") }, - new TimestampSpec("ds", "auto", null), + new TimestampSpec("ds", "auto", null, null), Granularities.ALL, Boolean.FALSE ); @@ -138,7 +138,7 @@ public void testMerge() final Metadata merged4 = new Metadata( Collections.singletonMap("k", "v"), explicitAggs, - new TimestampSpec("ds", "auto", null), + new TimestampSpec("ds", "auto", null, null), Granularities.ALL, null ); diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index 72b52873386f..0ec7f61b3624 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -275,7 +275,7 @@ public static IncrementalIndex makeRealtimeIndex(final CharSource source, boolea { final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) - .withTimestampSpec(new TimestampSpec("ds", "auto", null)) + .withTimestampSpec(new TimestampSpec("ds", "auto", null, null)) .withDimensionsSpec(DIMENSIONS_SPEC) .withVirtualColumns(VIRTUAL_COLUMNS) .withMetrics(METRIC_AGGS) @@ -306,7 +306,7 @@ public static IncrementalIndex loadIncrementalIndex( { final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "iso", null), + new TimestampSpec("ts", "iso", null, null), new DimensionsSpec(DIMENSION_SCHEMAS, null, null), "\t", "\u0001", diff --git a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java index c469c31356e2..00e2a550b076 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java @@ -51,7 +51,7 @@ public class AndFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java index 6e0a3cf050c0..f41fbd0adff8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java @@ -54,7 +54,7 @@ public class BoundFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java index a57d36f731eb..dd9850892e01 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java @@ -55,7 +55,7 @@ public class ColumnComparisonFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java index 3d3d50c971a2..d14236fa4785 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java @@ -59,7 +59,7 @@ public class ExpressionFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( ImmutableList.of( new StringDimensionSchema("dim0"), diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index 1d48b377338c..81cd6e069920 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -154,7 +154,7 @@ public DruidDoublePredicate makeDoublePredicate() private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java index 797ffe68c316..109d875fcd1c 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java @@ -79,7 +79,7 @@ public class FloatAndDoubleFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), new DimensionsSpec( ImmutableList.of( new StringDimensionSchema("dim0"), diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java index 77aa41ff9c2c..1d477402ff73 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java @@ -59,7 +59,7 @@ public class InFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java index bbd3fd7271e5..b469f28aec2a 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java @@ -55,7 +55,7 @@ public class InvalidFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java index f06da4f4494b..89c17d154531 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java @@ -55,7 +55,7 @@ public class JavaScriptFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java index b6ac4f040465..e0a290d4be52 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java @@ -51,7 +51,7 @@ public class LikeFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java index 94c5fbde17cb..2c8655d5ae26 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java @@ -76,7 +76,7 @@ public class LongFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java index f512d708d5c4..e2711ffc6f5e 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java @@ -50,7 +50,7 @@ public class NotFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java index 26de14da396a..b1c079537bcb 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java @@ -53,7 +53,7 @@ public class RegexFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java index 57604319fea3..c0ec17c113bc 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java @@ -55,7 +55,7 @@ public class SearchQueryFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java index 55f8c09b9c94..96063818de1c 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java @@ -59,7 +59,7 @@ public class SelectorFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3", "dim6")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java index 1b192f1db7c5..71234e6273d9 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java @@ -70,7 +70,7 @@ public class TimeFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java index a80c9103b42d..347e66362f4a 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java @@ -53,7 +53,7 @@ public void test() throws IndexSizeExceededException ); IncrementalIndexSchema schema = new IncrementalIndexSchema( 0, - new TimestampSpec("ds", "auto", null), + new TimestampSpec("ds", "auto", null, null), Granularities.ALL, VirtualColumns.EMPTY, dimensionsSpec, diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 6c5c5d490c2a..e8e10d1c699a 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -66,7 +66,7 @@ public void testDefaultExclusions() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null), + new TimestampSpec("time", "auto", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimB", "dimA")), null, null), null, null @@ -99,7 +99,7 @@ public void testExplicitInclude() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null), + new TimestampSpec("time", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), ImmutableList.of("dimC"), @@ -136,7 +136,7 @@ public void testTransformSpec() Map parserMap = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null), + new TimestampSpec("time", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), ImmutableList.of(), @@ -193,7 +193,7 @@ public void testOverlapMetricNameAndDim() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null), + new TimestampSpec("time", "auto", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( "time", "dimA", @@ -227,7 +227,7 @@ public void testDuplicateAggregators() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null), + new TimestampSpec("time", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time")), ImmutableList.of("dimC"), @@ -292,7 +292,7 @@ public void testEmptyDatasource() throws Exception Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null), + new TimestampSpec("time", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), ImmutableList.of("dimC"), @@ -355,7 +355,7 @@ public void testSerde() throws Exception Assert.assertEquals( actual.getParser().getParseSpec(), new JSONParseSpec( - new TimestampSpec("xXx", null, null), + new TimestampSpec("xXx", null, null, null), new DimensionsSpec(null, Arrays.asList("metric1", "xXx", "col1"), null), null, null diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java index 8102a719e8e6..7ba22c4e0f50 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java @@ -44,7 +44,7 @@ public class TransformSpecTest { private static final MapInputRowParser PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", DateTimes.of("2000-01-01")), + new TimestampSpec("t", "auto", DateTimes.of("2000-01-01"), null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("f", "x", "y")), null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java index 9c373bf97daf..d53d40878070 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java @@ -81,6 +81,7 @@ public void testSerde() throws Exception new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java index 3b6a9d92c474..e202465d3a29 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -121,7 +121,7 @@ public AppenderatorTester( final Map parserMap = objectMapper.convertValue( new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "auto", null), + new TimestampSpec("ts", "auto", null, null), new DimensionsSpec(null, null, null), null, null diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index fbf85b446001..b1ce9771db18 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -112,7 +112,7 @@ public int columnCacheSizeBytes() final Map parserMap = objectMapper.convertValue( new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "auto", null), + new TimestampSpec("ts", "auto", null, null), new DimensionsSpec(null, null, null), null, null diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java index 8d2232aad6bd..760420b33c9c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java @@ -74,6 +74,7 @@ public void setUp() throws Exception new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1")), null, null), null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java index 45b45bbeaed8..ff0a9c024909 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -90,6 +90,7 @@ public void setUp() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1")), null, null), null, @@ -234,6 +235,7 @@ public void testDuplicateRegistering() new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1")), null, null), null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index 1bb58ab40905..ed5742d58b41 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -204,7 +204,7 @@ private void createTestIndex(File segmentDir) throws Exception final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), + new TimestampSpec("timestamp", "yyyyMMddHH", null, null), DIMENSIONS_SPEC, "\t", null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java index 9930e8426af3..ba146e6d1573 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java @@ -75,6 +75,7 @@ public void testConnect() throws IOException new TimestampSpec( "timestamp", "auto", + null, null ), new DimensionsSpec( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java index 6626c867a990..08cd5e3b9dee 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java @@ -69,7 +69,7 @@ public class SqlFirehoseFactoryTest private final MapInputRowParser parser = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")), new ArrayList<>(), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 4052cc678f4a..ccb06bd76c89 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -137,7 +137,7 @@ public void setUp() throws Exception jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec(null, null, null), null, null @@ -157,7 +157,7 @@ public void setUp() throws Exception jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null), + new TimestampSpec("timestamp", "auto", null, null), new DimensionsSpec(null, null, null), null, null diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java index c251d31ee6d1..8c1d6a51356d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java @@ -63,8 +63,8 @@ */ public class Calcites { - private static final DateTimes.UtcFormatter CALCITE_DATE_PARSER = DateTimes.wrapFormatter(ISODateTimeFormat.dateParser()); - private static final DateTimes.UtcFormatter CALCITE_TIMESTAMP_PARSER = DateTimes.wrapFormatter( + private static final DateTimes.UtcFormatter CALCITE_DATE_PARSER = DateTimes.wrapUtcFormatter(ISODateTimeFormat.dateParser()); + private static final DateTimes.UtcFormatter CALCITE_TIMESTAMP_PARSER = DateTimes.wrapUtcFormatter( new DateTimeFormatterBuilder() .append(ISODateTimeFormat.dateParser()) .appendLiteral(' ') diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 9ca3233f20d9..8c017d1e72e5 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -242,7 +242,7 @@ public AuthenticationResult createEscalatedAuthenticationResult() private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")), null, From 4ab1a76d305332fb46c5177218eff23ea9e15886 Mon Sep 17 00:00:00 2001 From: zhaojiandong Date: Fri, 18 Jan 2019 16:48:46 +0800 Subject: [PATCH 2/6] add a doc --- docs/content/ingestion/ingestion-spec.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/ingestion/ingestion-spec.md b/docs/content/ingestion/ingestion-spec.md index 46e7334aeffa..f4ee57771500 100644 --- a/docs/content/ingestion/ingestion-spec.md +++ b/docs/content/ingestion/ingestion-spec.md @@ -208,6 +208,7 @@ handle all formatting decisions on their own, without using the ParseSpec. |-------|------|-------------|----------| | column | String | The column of the timestamp. | yes | | format | String | iso, posix, millis, micro, nano, auto or any [Joda time](http://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) format. | no (default == 'auto' | +| timeZone | String | Specify timeZone of the timestamp. e.g. "timestampSpec": {"format": "yyyy-MM-dd HH:mm:ss","column": "logTime","timestamp": "America/Los_Angeles"} | no (default == 'UTC') | From 0c0ba3a407151e248109c1568b8678c45dfa51f5 Mon Sep 17 00:00:00 2001 From: zhaojiandong Date: Fri, 15 Feb 2019 10:46:58 +0800 Subject: [PATCH 3/6] add VisibleForTesting to TimestampSpec and optimize the doc --- .../benchmark/FlattenJSONBenchmarkUtil.java | 23 +++++++++++++++---- .../druid/data/input/impl/TimestampSpec.java | 19 ++++++++++++++- .../druid/java/util/common/DateTimes.java | 2 +- .../data/input/impl/CSVParseSpecTest.java | 2 -- .../input/impl/DelimitedParseSpecTest.java | 5 +--- .../input/impl/FileIteratingFirehoseTest.java | 2 +- .../input/impl/InputRowParserSerdeTest.java | 10 ++++---- .../impl/JSONLowercaseParseSpecTest.java | 1 - .../data/input/impl/JSONParseSpecTest.java | 6 ++--- .../input/impl/JavaScriptParseSpecTest.java | 6 ++--- .../druid/data/input/impl/ParseSpecTest.java | 4 ---- .../data/input/impl/RegexParseSpecTest.java | 2 +- .../data/input/impl/SqlFirehoseTest.java | 2 +- .../input/impl/StringInputRowParserTest.java | 4 ++-- .../input/impl/TimeAndDimsParseSpecTest.java | 2 +- .../data/input/impl/TimestampSpecTest.java | 6 ++--- ...fetchableTextFilesFirehoseFactoryTest.java | 1 - docs/content/ingestion/ingestion-spec.md | 2 +- .../orc/OrcHadoopInputRowParserTest.java | 4 +--- .../input/orc/OrcIndexGeneratorJobTest.java | 2 +- .../thrift/ThriftInputRowParserTest.java | 4 ++-- .../segment/MapVirtualColumnSelectTest.java | 2 +- .../segment/MapVirtualColumnTestBase.java | 2 +- .../input/AvroStreamInputRowParserTest.java | 4 ++-- .../query/filter/BloomDimFilterTest.java | 2 +- .../indexing/kafka/KafkaIndexTaskTest.java | 2 +- .../kafka/supervisor/KafkaSupervisorTest.java | 2 +- .../kinesis/KinesisIndexTaskTest.java | 2 +- .../supervisor/KinesisSupervisorTest.java | 2 +- .../protobuf/ProtobufInputRowParserTest.java | 4 ++-- .../common/task/CompactionTaskRunTest.java | 1 - .../apache/druid/query/DoubleStorageTest.java | 2 +- .../druid/query/MultiValuedDimensionTest.java | 2 +- .../druid/query/SchemaEvolutionTest.java | 2 +- .../GroupByQueryRunnerFactoryTest.java | 2 +- .../metadata/SegmentMetadataQueryTest.java | 2 +- .../apache/druid/segment/MetadataTest.java | 10 ++++---- .../org/apache/druid/segment/TestIndex.java | 4 ++-- .../druid/segment/filter/AndFilterTest.java | 2 +- .../druid/segment/filter/BoundFilterTest.java | 2 +- .../filter/ColumnComparisonFilterTest.java | 2 +- .../segment/filter/ExpressionFilterTest.java | 2 +- .../segment/filter/FilterPartitionTest.java | 2 +- .../druid/segment/filter/InFilterTest.java | 2 +- .../segment/filter/InvalidFilteringTest.java | 2 +- .../segment/filter/JavaScriptFilterTest.java | 2 +- .../druid/segment/filter/LikeFilterTest.java | 2 +- .../druid/segment/filter/NotFilterTest.java | 2 +- .../druid/segment/filter/RegexFilterTest.java | 2 +- .../segment/filter/SearchQueryFilterTest.java | 2 +- .../segment/filter/SelectorFilterTest.java | 2 +- .../segment/filter/TimeFilteringTest.java | 2 +- .../IncrementalIndexMultiValueSpecTest.java | 2 +- .../segment/indexing/DataSchemaTest.java | 14 +++++------ .../segment/indexing/TransformSpecTest.java | 2 +- .../segment/realtime/FireDepartmentTest.java | 1 - .../appenderator/AppenderatorTester.java | 2 +- ...DefaultOfflineAppenderatorFactoryTest.java | 2 +- .../EventReceiverFirehoseIdleTest.java | 1 - .../firehose/EventReceiverFirehoseTest.java | 2 -- .../firehose/IngestSegmentFirehoseTest.java | 2 +- .../firehose/LocalFirehoseFactoryTest.java | 1 - .../firehose/SqlFirehoseFactoryTest.java | 2 +- .../plumber/RealtimePlumberSchoolTest.java | 4 ++-- .../druid/sql/calcite/util/CalciteTests.java | 18 +++++++-------- 65 files changed, 124 insertions(+), 113 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java index d1612982b345..4493a6581757 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java @@ -54,7 +54,7 @@ public FlattenJSONBenchmarkUtil() public Parser getFlatParser() { JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(null, null, null), null, null @@ -249,10 +249,23 @@ public String generateNestedEvent() throws Exception String[] dimsArray1 = {String.valueOf(rng.nextInt()), String.valueOf(rng.nextInt()), String.valueOf(rng.nextInt())}; BenchmarkEvent nestedDims2 = new BenchmarkEvent( null, - null, null, String.valueOf(rng.nextInt()), String.valueOf(rng.nextInt()), String.valueOf(rng.nextInt()), String.valueOf(rng.nextInt()), - null, null, null, null, - null, null, null, null, - dimsArray1, null, null + null, + null, + String.valueOf(rng.nextInt()), + String.valueOf(rng.nextInt()), + String.valueOf(rng.nextInt()), + String.valueOf(rng.nextInt()), + null, + null, + null, + null, + null, + null, + null, + null, + dimsArray1, + null, + null ); Long[] metricsArray1 = {rng.nextLong(), rng.nextLong(), rng.nextLong(), rng.nextLong()}; diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java index ea7ef852a0df..0f29c5b213b3 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import org.apache.commons.lang3.StringUtils; import org.apache.druid.guice.annotations.PublicApi; @@ -71,7 +72,23 @@ public TimestampSpec( { this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn; this.timestampFormat = format == null ? DEFAULT_FORMAT : format; - this.timeZone = (StringUtils.isBlank(timeZone)) ? DateTimes.UTC_TIMEZONE : timeZone; + this.timeZone = StringUtils.isBlank(timeZone) ? DateTimes.UTC_TIMEZONE : timeZone; + this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat, this.timeZone); + this.missingValue = missingValue == null + ? DEFAULT_MISSING_VALUE + : missingValue; + } + + @VisibleForTesting + public TimestampSpec( + @JsonProperty("column") String timestampColumn, + @JsonProperty("format") String format, + @JsonProperty("missingValue") DateTime missingValue + ) + { + this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn; + this.timestampFormat = format == null ? DEFAULT_FORMAT : format; + this.timeZone = DateTimes.UTC_TIMEZONE; this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat, this.timeZone); this.missingValue = missingValue == null ? DEFAULT_MISSING_VALUE diff --git a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java index 1c70b17c2f07..6da70dc985c7 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java +++ b/core/src/main/java/org/apache/druid/java/util/common/DateTimes.java @@ -79,7 +79,7 @@ public Formatter(DateTimeFormatter innerFormatter) public Formatter(String format, String timezone) { this.innerFormatter = DateTimeFormat.forPattern(format) - .withChronology(ISOChronology.getInstance(inferTzfromString(timezone))); + .withChronology(ISOChronology.getInstance(inferTzFromString(timezone))); } @SuppressForbidden(reason = "DateTimeFormatter#parseDateTime") diff --git a/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java index 50d951c274cc..f5fa4ee1b9bd 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/CSVParseSpecTest.java @@ -35,7 +35,6 @@ public void testColumnMissing() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -58,7 +57,6 @@ public void testComma() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java index b7f5332d9646..9b0786228d66 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/DelimitedParseSpecTest.java @@ -37,7 +37,7 @@ public class DelimitedParseSpecTest public void testSerde() throws IOException { DelimitedParseSpec spec = new DelimitedParseSpec( - new TimestampSpec("abc", "iso", null, null), + new TimestampSpec("abc", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "\u0001", "\u0002", @@ -66,7 +66,6 @@ public void testColumnMissing() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -90,7 +89,6 @@ public void testComma() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -114,7 +112,6 @@ public void testDefaultColumnList() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java b/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java index 7079411ad56d..7683258f3028 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/FileIteratingFirehoseTest.java @@ -77,7 +77,7 @@ public FileIteratingFirehoseTest(List texts, int numSkipHeaderRows) { parser = new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("ts", "auto", null, null), + new TimestampSpec("ts", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null), ",", ImmutableList.of("ts", "x"), diff --git a/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java b/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java index db906ed68a54..e79953ce4a6a 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/InputRowParserSerdeTest.java @@ -49,7 +49,7 @@ public void testStringInputRowParserSerde() throws Exception { final StringInputRowParser parser = new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), null, null), null, null @@ -91,7 +91,7 @@ public void testMapInputRowParserSerde() throws Exception { final MapInputRowParser parser = new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("timeposix", "posix", null, null), + new TimestampSpec("timeposix", "posix", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), ImmutableList.of("baz"), @@ -124,7 +124,7 @@ public void testMapInputRowParserNumbersSerde() throws Exception { final MapInputRowParser parser = new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("timemillis", "millis", null, null), + new TimestampSpec("timemillis", "millis", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "values")), ImmutableList.of("toobig", "value"), @@ -165,7 +165,7 @@ private InputRow testCharsetParseHelper(Charset charset) throws Exception { final StringInputRowParser parser = new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo", "bar")), null, null), null, null @@ -205,7 +205,7 @@ public void testFlattenParse() throws Exception JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); final StringInputRowParser parser = new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(null, null, null), flattenSpec, null diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java index 7c4d90fd68f6..72c320f4939b 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JSONLowercaseParseSpecTest.java @@ -36,7 +36,6 @@ public void testLowercasing() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java index d1f0f6db6344..4dd13b601a69 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JSONParseSpecTest.java @@ -44,7 +44,7 @@ public class JSONParseSpecTest public void testParseRow() { final JSONParseSpec parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), null, null), new JSONPathSpec( true, @@ -85,7 +85,7 @@ public void testParseRow() public void testParseRowWithConditional() { final JSONParseSpec parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("foo")), null, null), new JSONPathSpec( true, @@ -116,7 +116,7 @@ public void testSerde() throws IOException HashMap feature = new HashMap(); feature.put("ALLOW_UNQUOTED_CONTROL_CHARS", true); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("bar", "foo")), null, null), null, feature diff --git a/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java index 7bbb4f163330..ac6ecbad9e48 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/JavaScriptParseSpecTest.java @@ -53,7 +53,7 @@ public void testSerde() throws IOException ) ); JavaScriptParseSpec spec = new JavaScriptParseSpec( - new TimestampSpec("abc", "iso", null, null), + new TimestampSpec("abc", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "abc", JavaScriptConfig.getEnabledInstance() @@ -74,7 +74,7 @@ public void testMakeParser() { final JavaScriptConfig config = JavaScriptConfig.getEnabledInstance(); JavaScriptParseSpec spec = new JavaScriptParseSpec( - new TimestampSpec("abc", "iso", null, null), + new TimestampSpec("abc", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "function(str) { var parts = str.split(\"-\"); return { one: parts[0], two: parts[1] } }", config @@ -90,7 +90,7 @@ public void testMakeParserNotAllowed() { final JavaScriptConfig config = new JavaScriptConfig(false); JavaScriptParseSpec spec = new JavaScriptParseSpec( - new TimestampSpec("abc", "iso", null, null), + new TimestampSpec("abc", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "abc", config diff --git a/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java index e40f7e9a71bc..1c2e7808e174 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/ParseSpecTest.java @@ -67,7 +67,6 @@ public void testDuplicateNames() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -91,7 +90,6 @@ public void testDimAndDimExcluOverlap() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -115,7 +113,6 @@ public void testDimExclusionDuplicate() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -162,7 +159,6 @@ public void testDimensionSpecRequired() new TimestampSpec( "timestamp", "auto", - null, null ), null, diff --git a/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java index 2840945f9dbb..c31c6f9dbe3d 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/RegexParseSpecTest.java @@ -37,7 +37,7 @@ public class RegexParseSpecTest public void testSerde() throws IOException { RegexParseSpec spec = new RegexParseSpec( - new TimestampSpec("abc", "iso", null, null), + new TimestampSpec("abc", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Collections.singletonList("abc")), null, null), "\u0001", Collections.singletonList("abc"), diff --git a/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java b/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java index 05a5c232b8f5..f1c3d62481ae 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/SqlFirehoseTest.java @@ -81,7 +81,7 @@ public void setup() throws IOException this.fileList = testFile; parser = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("x")), null, null) ) ); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java b/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java index d26266a305c4..332e76f101c4 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/StringInputRowParserTest.java @@ -35,7 +35,7 @@ public class StringInputRowParserTest public void testDisableJavaScript() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( @@ -61,7 +61,7 @@ public void testDisableJavaScript() public void testDisableJavaScript2() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( diff --git a/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java index 3d5267ffa6b0..656a87d76706 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/TimeAndDimsParseSpecTest.java @@ -63,7 +63,7 @@ public void testSerdeWithNonNulls() throws Exception Assert.assertEquals( new TimeAndDimsParseSpec( - new TimestampSpec("tcol", null, null, null), + new TimestampSpec("tcol", null, null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null) ), actual diff --git a/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java b/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java index f5374509dd97..73201f79ff52 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/TimestampSpecTest.java @@ -31,7 +31,7 @@ public class TimestampSpecTest @Test public void testExtractTimestamp() { - TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null, null); + TimestampSpec spec = new TimestampSpec("TIMEstamp", "yyyy-MM-dd", null); Assert.assertEquals( DateTimes.of("2014-03-01"), spec.extractTimestamp(ImmutableMap.of("TIMEstamp", "2014-03-01")) @@ -41,7 +41,7 @@ public void testExtractTimestamp() @Test public void testExtractTimestampWithMissingTimestampColumn() { - TimestampSpec spec = new TimestampSpec(null, null, DateTimes.EPOCH, null); + TimestampSpec spec = new TimestampSpec(null, null, DateTimes.EPOCH); Assert.assertEquals( DateTimes.of("1970-01-01"), spec.extractTimestamp(ImmutableMap.of("dim", "foo")) @@ -59,7 +59,7 @@ public void testContextualTimestampList() "2000-01-01T05:00:02", "2000-01-01T05:00:03", }; - TimestampSpec spec = new TimestampSpec("TIMEstamp", DATE_FORMAT, null, null); + TimestampSpec spec = new TimestampSpec("TIMEstamp", DATE_FORMAT, null); DateTimes.UtcFormatter formatter = DateTimes.wrapUtcFormatter(ISODateTimeFormat.dateHourMinuteSecond()); diff --git a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java index 88b44db775da..c1d1b318d00d 100644 --- a/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java +++ b/core/src/test/java/org/apache/druid/data/input/impl/prefetch/PrefetchableTextFilesFirehoseFactoryTest.java @@ -69,7 +69,6 @@ public class PrefetchableTextFilesFirehoseFactoryTest new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( diff --git a/docs/content/ingestion/ingestion-spec.md b/docs/content/ingestion/ingestion-spec.md index f4ee57771500..4377eb1d46bf 100644 --- a/docs/content/ingestion/ingestion-spec.md +++ b/docs/content/ingestion/ingestion-spec.md @@ -208,7 +208,7 @@ handle all formatting decisions on their own, without using the ParseSpec. |-------|------|-------------|----------| | column | String | The column of the timestamp. | yes | | format | String | iso, posix, millis, micro, nano, auto or any [Joda time](http://joda-time.sourceforge.net/apidocs/org/joda/time/format/DateTimeFormat.html) format. | no (default == 'auto' | -| timeZone | String | Specify timeZone of the timestamp. e.g. "timestampSpec": {"format": "yyyy-MM-dd HH:mm:ss","column": "logTime","timestamp": "America/Los_Angeles"} | no (default == 'UTC') | +| timeZone | String | Specify timeZone of the timestamp, refer to the [Joda timezone list](http://joda-time.sourceforge.net/timezones.html) | no (default == 'UTC') | diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java index 4fcc546cd8c5..3b7f56928bb9 100644 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcHadoopInputRowParserTest.java @@ -108,7 +108,6 @@ public void testSerde() throws IOException new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -131,7 +130,6 @@ public void testTypeFromParseSpec() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( @@ -152,7 +150,7 @@ public void testParse() final String typeString = "struct,col3:float,col4:bigint,col5:decimal,col6:array,col7:map>"; final OrcHadoopInputRowParser parser = new OrcHadoopInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(null, null, null) ), typeString, diff --git a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java index 6798bcbc2abd..fd9b9b6ee467 100644 --- a/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/org/apache/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -132,7 +132,7 @@ public class OrcIndexGeneratorJobTest }; private final InputRowParser inputRowParser = new OrcHadoopInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null) ), "struct", diff --git a/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java b/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java index eebf87a61677..3d890a7c9ffb 100644 --- a/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java +++ b/extensions-contrib/thrift-extensions/src/test/java/org/apache/druid/data/input/thrift/ThriftInputRowParserTest.java @@ -59,7 +59,7 @@ public class ThriftInputRowParserTest @Before public void setUp() { - parseSpec = new JSONParseSpec(new TimestampSpec("date", "auto", null, null), + parseSpec = new JSONParseSpec(new TimestampSpec("date", "auto", null), new DimensionsSpec(Lists.newArrayList( new StringDimensionSchema("title"), new StringDimensionSchema("lastName") @@ -121,7 +121,7 @@ public void testParse() throws Exception public void testDisableJavaScript() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java index df5dd767f4c9..f248bec0d3bb 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnSelectTest.java @@ -87,7 +87,7 @@ public static Iterable constructorFeeder() throws IOException final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("dim", "keys", "values")), null, null), "\t", ",", diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java index c09f104b91bb..ffc3706f95e5 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java @@ -45,7 +45,7 @@ static IncrementalIndex generateIndex() throws IOException final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "auto", null, null), + new TimestampSpec("ts", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("dim", "keys", "values")), null, null), "\t", ",", diff --git a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java index 65a237a9c787..184c8b247023 100644 --- a/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java +++ b/extensions-core/avro-extensions/src/test/java/org/apache/druid/data/input/AvroStreamInputRowParserTest.java @@ -99,7 +99,7 @@ public class AvroStreamInputRowParserTest "timestamp" ); public static final AvroParseSpec PARSE_SPEC = new AvroParseSpec( - new TimestampSpec("nested", "millis", null, null), + new TimestampSpec("nested", "millis", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(DIMENSIONS), Collections.emptyList(), null), new JSONPathSpec( true, @@ -109,7 +109,7 @@ public class AvroStreamInputRowParserTest ) ); public static final AvroParseSpec PARSE_SPEC_SCHEMALESS = new AvroParseSpec( - new TimestampSpec("nested", "millis", null, null), + new TimestampSpec("nested", "millis", null), new DimensionsSpec(null, null, null), new JSONPathSpec( true, diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java index d8c4e7556193..fb8d31aa4a78 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java @@ -61,7 +61,7 @@ public class BloomDimFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3", "dim6")), null, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index dba2ac48d064..b7b389668a46 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -235,7 +235,7 @@ public KafkaIndexTaskTest(boolean isIncrementalHandoffSupported) objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec( Arrays.asList( new StringDimensionSchema("dim1"), diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 6bec67b17df2..a08806acc173 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -2860,7 +2860,7 @@ private static DataSchema getDataSchema(String dataSource) objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec( dimensions, null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 40009b284008..a8130b9c0e5b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -254,7 +254,7 @@ public class KinesisIndexTaskTest extends EasyMockSupport objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec( Arrays.asList( new StringDimensionSchema("dim1"), diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index f698ef9b8d4a..750b9247d792 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -3536,7 +3536,7 @@ private static DataSchema getDataSchema(String dataSource) objectMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec( dimensions, null, diff --git a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java index f0d4939293e1..081fc23a24f6 100644 --- a/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java +++ b/extensions-core/protobuf-extensions/src/test/java/org/apache/druid/data/input/protobuf/ProtobufInputRowParserTest.java @@ -58,7 +58,7 @@ public class ProtobufInputRowParserTest public void setUp() { parseSpec = new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(Lists.newArrayList( new StringDimensionSchema("event"), new StringDimensionSchema("id"), @@ -181,7 +181,7 @@ public void testParse() throws Exception public void testDisableJavaScript() { final JavaScriptParseSpec parseSpec = new JavaScriptParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas( ImmutableList.of( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index a7c87cd5c4b7..54796a542fd2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -82,7 +82,6 @@ public class CompactionTaskRunTest extends IngestionTestBase new TimestampSpec( "ts", "auto", - null, null ), new DimensionsSpec( diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index 8e0c2b6e2311..a85b41d61165 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -121,7 +121,7 @@ private Druids.ScanQueryBuilder newTestQuery() private static final InputRowParser> ROW_PARSER = new MapInputRowParser( new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null, null), + new TimestampSpec(TIME_COLUMN, "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of(DIM_NAME)), ImmutableList.of(DIM_FLOAT_NAME), diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index 4b0770ebf165..c4c4a0b99af6 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -121,7 +121,7 @@ public void setup() throws Exception StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("product", "tags")), null, null), "\t", ImmutableList.of("timestamp", "product", "tags"), diff --git a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java index d90eeb0f1840..c15a2e60deb8 100644 --- a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java @@ -79,7 +79,7 @@ public static List inputRowsWithDimensions(final List dimensio { final MapInputRowParser parser = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", null, null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(dimensions), dimensions.isEmpty() ? ImmutableList.of("t", "c1", "c2") : null, diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index a8fe64263a99..56598e41d6f2 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -149,7 +149,7 @@ private Segment createSegment() throws Exception StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "iso", null, null), + new TimestampSpec("timestamp", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("product", "tags")), null, null), "\t", ImmutableList.of("timestamp", "product", "tags"), diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java index 0f2df13e7408..31e70d376644 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentMetadataQueryTest.java @@ -719,7 +719,7 @@ public void testSegmentMetadataQueryWithTimestampSpecMerge() 0, expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(), null, - new TimestampSpec("ds", "auto", null, null), + new TimestampSpec("ds", "auto", null), null, null ); diff --git a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java index ec65cf5ab070..c763e66b1ed4 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -81,7 +81,7 @@ public void testMerge() final Metadata m1 = new Metadata( Collections.singletonMap("k", "v"), aggs, - new TimestampSpec("ds", "auto", null, null), + new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE ); @@ -89,7 +89,7 @@ public void testMerge() final Metadata m2 = new Metadata( Collections.singletonMap("k", "v"), aggs, - new TimestampSpec("ds", "auto", null, null), + new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE ); @@ -97,7 +97,7 @@ public void testMerge() final Metadata m3 = new Metadata( Collections.singletonMap("k", "v"), aggs, - new TimestampSpec("ds", "auto", null, null), + new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.TRUE ); @@ -107,7 +107,7 @@ public void testMerge() new AggregatorFactory[]{ new LongMaxAggregatorFactory("n", "n") }, - new TimestampSpec("ds", "auto", null, null), + new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE ); @@ -138,7 +138,7 @@ public void testMerge() final Metadata merged4 = new Metadata( Collections.singletonMap("k", "v"), explicitAggs, - new TimestampSpec("ds", "auto", null, null), + new TimestampSpec("ds", "auto", null), Granularities.ALL, null ); diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index 0ec7f61b3624..72b52873386f 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -275,7 +275,7 @@ public static IncrementalIndex makeRealtimeIndex(final CharSource source, boolea { final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) - .withTimestampSpec(new TimestampSpec("ds", "auto", null, null)) + .withTimestampSpec(new TimestampSpec("ds", "auto", null)) .withDimensionsSpec(DIMENSIONS_SPEC) .withVirtualColumns(VIRTUAL_COLUMNS) .withMetrics(METRIC_AGGS) @@ -306,7 +306,7 @@ public static IncrementalIndex loadIncrementalIndex( { final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(DIMENSION_SCHEMAS, null, null), "\t", "\u0001", diff --git a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java index 00e2a550b076..c469c31356e2 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/AndFilterTest.java @@ -51,7 +51,7 @@ public class AndFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java index f41fbd0adff8..6e0a3cf050c0 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java @@ -54,7 +54,7 @@ public class BoundFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java index dd9850892e01..a57d36f731eb 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java @@ -55,7 +55,7 @@ public class ColumnComparisonFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java index d14236fa4785..3d3d50c971a2 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java @@ -59,7 +59,7 @@ public class ExpressionFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( ImmutableList.of( new StringDimensionSchema("dim0"), diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index 81cd6e069920..1d48b377338c 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -154,7 +154,7 @@ public DruidDoublePredicate makeDoublePredicate() private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java index 1d477402ff73..77aa41ff9c2c 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java @@ -59,7 +59,7 @@ public class InFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java index b469f28aec2a..bbd3fd7271e5 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InvalidFilteringTest.java @@ -55,7 +55,7 @@ public class InvalidFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java index 89c17d154531..f06da4f4494b 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java @@ -55,7 +55,7 @@ public class JavaScriptFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java index e0a290d4be52..b6ac4f040465 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java @@ -51,7 +51,7 @@ public class LikeFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java index e2711ffc6f5e..f512d708d5c4 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/NotFilterTest.java @@ -50,7 +50,7 @@ public class NotFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec(null, null, null) ) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java index b1c079537bcb..26de14da396a 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java @@ -53,7 +53,7 @@ public class RegexFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java index c0ec17c113bc..57604319fea3 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java @@ -55,7 +55,7 @@ public class SearchQueryFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java index 96063818de1c..55f8c09b9c94 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java @@ -59,7 +59,7 @@ public class SelectorFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3", "dim6")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java index 71234e6273d9..1b192f1db7c5 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/TimeFilteringTest.java @@ -70,7 +70,7 @@ public class TimeFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java index 347e66362f4a..a80c9103b42d 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java @@ -53,7 +53,7 @@ public void test() throws IndexSizeExceededException ); IncrementalIndexSchema schema = new IncrementalIndexSchema( 0, - new TimestampSpec("ds", "auto", null, null), + new TimestampSpec("ds", "auto", null), Granularities.ALL, VirtualColumns.EMPTY, dimensionsSpec, diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index e8e10d1c699a..6c5c5d490c2a 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -66,7 +66,7 @@ public void testDefaultExclusions() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null, null), + new TimestampSpec("time", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimB", "dimA")), null, null), null, null @@ -99,7 +99,7 @@ public void testExplicitInclude() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null, null), + new TimestampSpec("time", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), ImmutableList.of("dimC"), @@ -136,7 +136,7 @@ public void testTransformSpec() Map parserMap = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null, null), + new TimestampSpec("time", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), ImmutableList.of(), @@ -193,7 +193,7 @@ public void testOverlapMetricNameAndDim() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null, null), + new TimestampSpec("time", "auto", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of( "time", "dimA", @@ -227,7 +227,7 @@ public void testDuplicateAggregators() Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null, null), + new TimestampSpec("time", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time")), ImmutableList.of("dimC"), @@ -292,7 +292,7 @@ public void testEmptyDatasource() throws Exception Map parser = jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("time", "auto", null, null), + new TimestampSpec("time", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), ImmutableList.of("dimC"), @@ -355,7 +355,7 @@ public void testSerde() throws Exception Assert.assertEquals( actual.getParser().getParseSpec(), new JSONParseSpec( - new TimestampSpec("xXx", null, null, null), + new TimestampSpec("xXx", null, null), new DimensionsSpec(null, Arrays.asList("metric1", "xXx", "col1"), null), null, null diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java index 7ba22c4e0f50..8102a719e8e6 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/TransformSpecTest.java @@ -44,7 +44,7 @@ public class TransformSpecTest { private static final MapInputRowParser PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", DateTimes.of("2000-01-01"), null), + new TimestampSpec("t", "auto", DateTimes.of("2000-01-01")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("f", "x", "y")), null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java index d53d40878070..9c373bf97daf 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java @@ -81,7 +81,6 @@ public void testSerde() throws Exception new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java index e202465d3a29..3b6a9d92c474 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -121,7 +121,7 @@ public AppenderatorTester( final Map parserMap = objectMapper.convertValue( new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "auto", null, null), + new TimestampSpec("ts", "auto", null), new DimensionsSpec(null, null, null), null, null diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index b1ce9771db18..fbf85b446001 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -112,7 +112,7 @@ public int columnCacheSizeBytes() final Map parserMap = objectMapper.convertValue( new MapInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "auto", null, null), + new TimestampSpec("ts", "auto", null), new DimensionsSpec(null, null, null), null, null diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java index 760420b33c9c..8d2232aad6bd 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseIdleTest.java @@ -74,7 +74,6 @@ public void setUp() throws Exception new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1")), null, null), null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java index ff0a9c024909..45b45bbeaed8 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/EventReceiverFirehoseTest.java @@ -90,7 +90,6 @@ public void setUp() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1")), null, null), null, @@ -235,7 +234,6 @@ public void testDuplicateRegistering() new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("d1")), null, null), null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index ed5742d58b41..1bb58ab40905 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -204,7 +204,7 @@ private void createTestIndex(File segmentDir) throws Exception final StringInputRowParser parser = new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), DIMENSIONS_SPEC, "\t", null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java index ba146e6d1573..9930e8426af3 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/LocalFirehoseFactoryTest.java @@ -75,7 +75,6 @@ public void testConnect() throws IOException new TimestampSpec( "timestamp", "auto", - null, null ), new DimensionsSpec( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java index 08cd5e3b9dee..6626c867a990 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java @@ -69,7 +69,7 @@ public class SqlFirehoseFactoryTest private final MapInputRowParser parser = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")), new ArrayList<>(), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index ccb06bd76c89..4052cc678f4a 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -137,7 +137,7 @@ public void setUp() throws Exception jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(null, null, null), null, null @@ -157,7 +157,7 @@ public void setUp() throws Exception jsonMapper.convertValue( new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("timestamp", "auto", null, null), + new TimestampSpec("timestamp", "auto", null), new DimensionsSpec(null, null, null), null, null diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 8c017d1e72e5..069cbac35365 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -242,7 +242,7 @@ public AuthenticationResult createEscalatedAuthenticationResult() private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "iso", null, null), + new TimestampSpec(TIMESTAMP_COLUMN, "iso", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim3")), null, @@ -426,7 +426,6 @@ public AuthenticationResult createEscalatedAuthenticationResult() ); - public static final List ROWS2 = ImmutableList.of( createRow("2000-01-01", "דרואיד", "he", 1.0), createRow("2000-01-01", "druid", "en", 1.0), @@ -647,13 +646,14 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .shardSpec(new LinearShardSpec(0)) .build(), forbiddenIndex - ).add(DataSegment.builder() - .dataSource(DATASOURCE3) - .interval(indexNumericDims.getDataInterval()) - .version("1") - .shardSpec(new LinearShardSpec(0)) - .build(), - indexNumericDims + ).add( + DataSegment.builder() + .dataSource(DATASOURCE3) + .interval(indexNumericDims.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .build(), + indexNumericDims ); } From 363f46d0743fc16d9bf76ca79665678fba229972 Mon Sep 17 00:00:00 2001 From: zhaojiandong Date: Fri, 15 Feb 2019 11:04:31 +0800 Subject: [PATCH 4/6] change constructor of TimestampSpec in test classes --- .../apache/druid/indexer/BatchDeltaIngestionTest.java | 2 +- .../indexer/DetermineHashedPartitionsJobTest.java | 2 +- .../druid/indexer/DeterminePartitionsJobTest.java | 2 +- .../druid/indexer/HadoopDruidIndexerMapperTest.java | 2 +- .../druid/indexer/IndexGeneratorCombinerTest.java | 2 +- .../apache/druid/indexer/IndexGeneratorJobTest.java | 2 +- .../java/org/apache/druid/indexer/JobHelperTest.java | 2 +- .../druid/indexer/path/DatasourcePathSpecTest.java | 2 +- .../druid/indexer/updater/HadoopConverterJobTest.java | 2 +- .../task/AppenderatorDriverRealtimeIndexTaskTest.java | 2 +- .../druid/indexing/common/task/IndexTaskTest.java | 10 ---------- .../indexing/common/task/RealtimeIndexTaskTest.java | 2 +- .../AbstractParallelIndexSupervisorTaskTest.java | 1 - .../parallel/ParallelIndexSupervisorTaskSerdeTest.java | 1 - .../firehose/IngestSegmentFirehoseFactoryTest.java | 4 ++-- .../IngestSegmentFirehoseFactoryTimelineTest.java | 2 +- .../segment/filter/FloatAndDoubleFilteringTest.java | 2 +- .../apache/druid/segment/filter/LongFilteringTest.java | 2 +- 18 files changed, 16 insertions(+), 28 deletions(-) diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index f5253940c590..b68272460472 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -447,7 +447,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "host2", "visited_num"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java index 4ea346d91ec6..bbec6766419d 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -146,7 +146,7 @@ public DetermineHashedPartitionsJobTest( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", null, null, null), + new TimestampSpec("ts", null, null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of( "market", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java index 078b4ae5de98..51f4120752de 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java @@ -224,7 +224,7 @@ public DeterminePartitionsJobTest( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")), null, null), null, ImmutableList.of("timestamp", "host", "country", "visited_num"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java index f65b652983c1..43ca77d6217b 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -64,7 +64,7 @@ public class HadoopDruidIndexerMapperTest JSON_MAPPER.convertValue( new HadoopyStringInputRowParser( new JSONParseSpec( - new TimestampSpec("t", "auto", null, null), + new TimestampSpec("t", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")), null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java index 4931dc7ab977..aeadd46adb66 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java @@ -69,7 +69,7 @@ public void setUp() throws Exception HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords")), null, null) ), null diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index 9d8443340376..f3041fef13d5 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -142,7 +142,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index a51607cce25b..4aa6a5904cd4 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -80,7 +80,7 @@ public void setup() throws Exception HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java index 3fbf0383542e..a229d8be348f 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java @@ -327,7 +327,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig() HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(null, null, null), null, ImmutableList.of("timestamp", "host", "visited"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java index fbae209654d5..f81c02a623be 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/updater/HadoopConverterJobTest.java @@ -161,7 +161,7 @@ public InputStream openStream() throws IOException HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( new StringInputRowParser( new DelimitedParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList(TestIndex.DIMENSIONS)), null, null), "\t", "\u0001", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 079cf6220f21..b57397ab8cd6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1378,7 +1378,7 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( TestHelper.makeJsonMapper().convertValue( new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", null, null), + new TimestampSpec("t", "auto", null), new DimensionsSpec( ImmutableList.of( new StringDimensionSchema("dim1"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 0c0ae758537b..ef7dbce7b8d4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -132,7 +132,6 @@ public class IndexTaskTest new TimestampSpec( "ts", "auto", - null, null ), new DimensionsSpec( @@ -668,7 +667,6 @@ public void testCSVFileWithHeader() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -721,7 +719,6 @@ public void testCSVFileWithHeaderColumnOverride() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -931,7 +928,6 @@ public void testIgnoreParseException() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -985,7 +981,6 @@ public void testReportParseException() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -1075,7 +1070,6 @@ public void testMultipleParseExceptionsSuccess() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -1199,7 +1193,6 @@ public void testMultipleParseExceptionsFailure() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -1316,7 +1309,6 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -1420,7 +1412,6 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( @@ -1491,7 +1482,6 @@ public void testCsvWithHeaderOfEmptyTimestamp() throws Exception new TimestampSpec( "time", "auto", - null, null ), new DimensionsSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 861e7eca298c..2c3d3c3d8730 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -893,7 +893,7 @@ private RealtimeIndexTask makeRealtimeTask( TestHelper.makeJsonMapper().convertValue( new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", null, null), + new TimestampSpec("t", "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim1t")), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 3789ace16935..5d42919fa49f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -82,7 +82,6 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase new TimestampSpec( "ts", "auto", - null, null ), new DimensionsSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index 5ed440a8014c..0fa747f6604f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -63,7 +63,6 @@ public class ParallelIndexSupervisorTaskSerdeTest new TimestampSpec( "ts", "auto", - null, null ), new DimensionsSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 5b30302f33e0..3329419cf337 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -326,7 +326,7 @@ public DataSegment restore(DataSegment segment) ROW_PARSER, new MapInputRowParser( new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null, null), + new TimestampSpec(TIME_COLUMN, "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of()), ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME), @@ -441,7 +441,7 @@ public IngestSegmentFirehoseFactoryTest( private static final InputRowParser> ROW_PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null, null), + new TimestampSpec(TIME_COLUMN, "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of(DIM_NAME)), ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 35def5ec2bb0..70e877bad45b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -99,7 +99,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest private static final InputRowParser> ROW_PARSER = TransformSpec.NONE.decorate( new MapInputRowParser( new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null, null), + new TimestampSpec(TIME_COLUMN, "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS)), null, diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java index 109d875fcd1c..797ffe68c316 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FloatAndDoubleFilteringTest.java @@ -79,7 +79,7 @@ public class FloatAndDoubleFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), new DimensionsSpec( ImmutableList.of( new StringDimensionSchema("dim0"), diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java index 2c8655d5ae26..94c5fbde17cb 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LongFilteringTest.java @@ -76,7 +76,7 @@ public class LongFilteringTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000"), null), + new TimestampSpec(TIMESTAMP_COLUMN, "millis", DateTimes.of("2000")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")), null, From da9d3569931e79bc5533bd61400bf5effc11ab3f Mon Sep 17 00:00:00 2001 From: zhaojiandong Date: Fri, 15 Feb 2019 11:13:53 +0800 Subject: [PATCH 5/6] change definition VisibleForTesting constructor of TimestampSpec --- .../druid/data/input/impl/TimestampSpec.java | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java index 0f29c5b213b3..9f9c2ec0f6a2 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/TimestampSpec.java @@ -81,18 +81,12 @@ public TimestampSpec( @VisibleForTesting public TimestampSpec( - @JsonProperty("column") String timestampColumn, - @JsonProperty("format") String format, - @JsonProperty("missingValue") DateTime missingValue + String timestampColumn, + String format, + DateTime missingValue ) { - this.timestampColumn = (timestampColumn == null) ? DEFAULT_COLUMN : timestampColumn; - this.timestampFormat = format == null ? DEFAULT_FORMAT : format; - this.timeZone = DateTimes.UTC_TIMEZONE; - this.timestampConverter = TimestampParser.createObjectTimestampParser(timestampFormat, this.timeZone); - this.missingValue = missingValue == null - ? DEFAULT_MISSING_VALUE - : missingValue; + this(timestampColumn, format, missingValue, null); } @JsonProperty("column") From 2b3073bf95babb763049e5bd3d81c2a098328afd Mon Sep 17 00:00:00 2001 From: zhaojiandong Date: Fri, 15 Feb 2019 11:25:35 +0800 Subject: [PATCH 6/6] change constructor calling of TimestampSpec in tests --- .../druid/benchmark/FlattenJSONBenchmarkUtil.java | 8 ++++---- .../apache/druid/indexer/IndexGeneratorJobTest.java | 10 +++++----- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java b/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java index 4493a6581757..72e28f8b4a91 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/FlattenJSONBenchmarkUtil.java @@ -68,7 +68,7 @@ public Parser getFieldDiscoveryParser() JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(null, null, null), flattenSpec, null @@ -111,7 +111,7 @@ public Parser getNestedParser() JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(null, null, null), flattenSpec, null @@ -154,7 +154,7 @@ public Parser getForcedPathParser() JSONPathSpec flattenSpec = new JSONPathSpec(false, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(null, null, null), flattenSpec, null @@ -195,7 +195,7 @@ public Parser getJqParser() JSONPathSpec flattenSpec = new JSONPathSpec(true, fields); JSONParseSpec spec = new JSONParseSpec( - new TimestampSpec("ts", "iso", null, null), + new TimestampSpec("ts", "iso", null), new DimensionsSpec(null, null, null), flattenSpec, null diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index f3041fef13d5..8e191c7fba91 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -191,7 +191,7 @@ public static Collection constructFeed() null, new HadoopyStringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), @@ -239,7 +239,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), @@ -298,7 +298,7 @@ public static Collection constructFeed() SequenceFileInputFormat.class.getName(), new HadoopyStringInputRowParser( new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null, null), + new TimestampSpec("timestamp", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host")), null, null), null, ImmutableList.of("timestamp", "host", "visited_num"), @@ -332,7 +332,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "yyyyMMddHH", null, null), + new TimestampSpec("ts", "yyyyMMddHH", null), new DimensionsSpec(null, null, null), null, null @@ -365,7 +365,7 @@ public static Collection constructFeed() null, new StringInputRowParser( new JSONParseSpec( - new TimestampSpec("ts", "yyyyMMddHH", null, null), + new TimestampSpec("ts", "yyyyMMddHH", null), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("B", "F", "M", "Q", "X", "Y")), null, null), null, null