From 02e0555dce4484b64f365e93d4dab7a715d6bc4f Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 29 Sep 2017 14:11:57 -0700 Subject: [PATCH 01/23] Introduce "transformSpec" at ingest-time. It accepts a "filter" (standard query filter object) and "transforms" (a list of objects with "name" and "expression"). These can be used to do filtering and single-row transforms without need for a separate data processing job. The "expression" fields use the same expression language as other expression-based feature. --- docs/content/misc/math-expr.md | 11 +- .../input/orc/OrcIndexGeneratorJobTest.java | 1 + .../indexing/kafka/KafkaIndexTaskTest.java | 1 + .../kafka/supervisor/KafkaSupervisorTest.java | 1 + .../indexer/BatchDeltaIngestionTest.java | 1 + .../DetermineHashedPartitionsJobTest.java | 1 + .../indexer/DeterminePartitionsJobTest.java | 1 + .../indexer/HadoopDruidIndexerConfigTest.java | 2 + ...cUpdateDatasourcePathSpecSegmentsTest.java | 1 + .../indexer/IndexGeneratorCombinerTest.java | 1 + .../java/io/druid/indexer/JobHelperTest.java | 1 + .../indexer/path/DatasourcePathSpecTest.java | 1 + .../indexer/path/GranularityPathSpecTest.java | 2 + .../indexer/path/StaticPathSpecTest.java | 2 +- .../updater/HadoopConverterJobTest.java | 1 + .../indexing/common/TestRealtimeTask.java | 2 +- .../indexing/common/task/IndexTaskTest.java | 1 + .../common/task/RealtimeIndexTaskTest.java | 1 + .../indexing/common/task/TaskSerdeTest.java | 4 + .../indexing/overlord/TaskLifecycleTest.java | 4 + .../indexing/worker/TaskAnnouncementTest.java | 2 +- .../io/druid/segment/indexing/DataSchema.java | 26 +- .../io/druid/segment/indexing/Transform.java | 95 +++++++ .../druid/segment/indexing/TransformSpec.java | 124 ++++++++++ .../druid/segment/indexing/Transformer.java | 231 ++++++++++++++++++ .../indexing/TransformingInputRowParser.java | 56 +++++ .../TransformingStringInputRowParser.java | 63 +++++ .../segment/indexing/DataSchemaTest.java | 67 +++++ .../segment/indexing/TransformSpecTest.java | 152 ++++++++++++ .../segment/realtime/FireDepartmentTest.java | 1 + .../segment/realtime/RealtimeManagerTest.java | 3 + .../appenderator/AppenderatorTester.java | 1 + ...DefaultOfflineAppenderatorFactoryTest.java | 1 + .../plumber/RealtimePlumberSchoolTest.java | 2 + .../segment/realtime/plumber/SinkTest.java | 1 + .../cli/validate/DruidJsonValidatorTest.java | 1 + 36 files changed, 852 insertions(+), 14 deletions(-) create mode 100644 server/src/main/java/io/druid/segment/indexing/Transform.java create mode 100644 server/src/main/java/io/druid/segment/indexing/TransformSpec.java create mode 100644 server/src/main/java/io/druid/segment/indexing/Transformer.java create mode 100644 server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java create mode 100644 server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java create mode 100644 server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java diff --git a/docs/content/misc/math-expr.md b/docs/content/misc/math-expr.md index 0ee6d78300b6..957c6fc2923b 100644 --- a/docs/content/misc/math-expr.md +++ b/docs/content/misc/math-expr.md @@ -13,13 +13,16 @@ This expression language supports the following operators (listed in decreasing |<, <=, >, >=, ==, !=|Binary Comparison| |&&,\|\||Binary Logical AND, OR| -Long, double and string data types are supported. If a number contains a dot, it is interpreted as a double, otherwise it is interpreted as a long. That means, always add a '.' to your number if you want it interpreted as a double value. String literal should be quoted by single quotation marks. +Long, double, and string data types are supported. If a number contains a dot, it is interpreted as a double, otherwise it is interpreted as a long. That means, always add a '.' to your number if you want it interpreted as a double value. String literals should be quoted by single quotation marks. -Expressions can contain variables. Variable names may contain letters, digits, '\_' and '$'. Variable names must not begin with a digit. To escape other special characters, user can quote it with double quotation marks. +Multi-value types are not fully supported yet. Expressions may behave inconsistently on multi-value types, and you +should not rely on the behavior in this case to stay the same in future releases. -For logical operators, a number is true if and only if it is positive (0 or minus value means false). For string type, it's evaluation result of 'Boolean.valueOf(string)'. +Expressions can contain variables. Variable names may contain letters, digits, '\_' and '$'. Variable names must not begin with a digit. To escape other special characters, you can quote it with double quotation marks. -Also, the following built-in functions are supported. +For logical operators, a number is true if and only if it is positive (0 or negative value means false). For string type, it's the evaluation result of 'Boolean.valueOf(string)'. + +The following built-in functions are available. ## General functions diff --git a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java index 4d2ae18e466f..d68f2e93dc55 100644 --- a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/OrcIndexGeneratorJobTest.java @@ -206,6 +206,7 @@ public void setUp() throws Exception new UniformGranularitySpec( Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval) ), + null, mapper ), new HadoopIOConfig( diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 3f7ba71ba1e7..a7435597461e 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -199,6 +199,7 @@ public class KafkaIndexTaskTest ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, objectMapper ); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index b01fd38268d7..fb64e5d5be4b 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -1892,6 +1892,7 @@ private static DataSchema getDataSchema(String dataSource) Granularities.NONE, ImmutableList.of() ), + null, objectMapper ); } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index 1834415036e2..e751e31282a9 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -363,6 +363,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig(Mapof("paths", "bar", "type", "static"), null, null), @@ -136,6 +137,7 @@ public void testNoneShardSpecBucketSelection() Granularities.MINUTE, ImmutableList.of(Intervals.of("2010-01-01/P1D")) ), + null, jsonMapper ), new HadoopIOConfig(ImmutableMap.of("paths", "bar", "type", "static"), null, null), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java index 6899ccb39542..d019689262c2 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java @@ -236,6 +236,7 @@ private HadoopDruidIndexerConfig testRunUpdateSegmentListIfDatasourcePathSpecIsU null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) ), + null, jsonMapper ), new HadoopIOConfig( diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java index 4e9a1afbb9df..471885c7039a 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorCombinerTest.java @@ -82,6 +82,7 @@ public void setUp() throws Exception new UniformGranularitySpec( Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of("2010/2011")) ), + null, HadoopDruidIndexerConfig.JSON_MAPPER ), new HadoopIOConfig( diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java index 5f3c15781c82..f292cfb96081 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/JobHelperTest.java @@ -89,6 +89,7 @@ public void setup() throws Exception new UniformGranularitySpec( Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval) ), + null, HadoopDruidIndexerConfig.JSON_MAPPER ), new HadoopIOConfig( diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java index 466b11b8ba9c..a13e7dd64861 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java @@ -279,6 +279,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig() new UniformGranularitySpec( Granularities.DAY, Granularities.NONE, ImmutableList.of(Intervals.of("2000/3000")) ), + null, HadoopDruidIndexerConfig.JSON_MAPPER ), new HadoopIOConfig( diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java index 9e3a0447530c..3aec576f4f56 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/GranularityPathSpecTest.java @@ -153,6 +153,7 @@ public void testAddInputPath() throws Exception Granularities.MINUTE, ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z")) ), + null, jsonMapper ), new HadoopIOConfig(null, null, null), @@ -204,6 +205,7 @@ public void testIntervalTrimming() throws Exception Granularities.ALL, ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z")) ), + null, jsonMapper ), new HadoopIOConfig(null, null, null), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/StaticPathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/StaticPathSpecTest.java index ae75d6637db3..59b8ed0e210e 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/StaticPathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/StaticPathSpecTest.java @@ -54,7 +54,7 @@ public void testAddingPaths() throws Exception Job job = new Job(); StaticPathSpec pathSpec = new StaticPathSpec("/a/c,/a/b/{c,d}", null); - DataSchema schema = new DataSchema("ds", null, new AggregatorFactory[0], null, jsonMapper); + DataSchema schema = new DataSchema("ds", null, new AggregatorFactory[0], null, null, jsonMapper); HadoopIOConfig io = new HadoopIOConfig(null, null, null); pathSpec.addInputPaths(new HadoopDruidIndexerConfig(new HadoopIngestionSpec(schema, io, null)), job); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index 61350ac9a882..2b7766f7f72d 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -182,6 +182,7 @@ public InputStream openStream() throws IOException Granularities.DAY, ImmutableList.of(interval) ), + null, HadoopDruidIndexerConfig.JSON_MAPPER ), new HadoopIOConfig( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java index f9cb242218ab..98db78655510 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestRealtimeTask.java @@ -54,7 +54,7 @@ public TestRealtimeTask( id, taskResource, new FireDepartment( - new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, mapper), + new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper), new RealtimeIOConfig( new LocalFirehoseFactory(new File("lol"), "rofl", null), (schema, config, metrics) -> null, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 029024a38bb9..f59a97d70426 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -1008,6 +1008,7 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( Granularities.MINUTE, Arrays.asList(Intervals.of("2014/2015")) ), + null, jsonMapper ), new IndexTask.IndexIOConfig( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 846f242ad2b4..c9f51ac96083 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -879,6 +879,7 @@ private RealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportPa null, new AggregatorFactory[]{new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("met1", "met1")}, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, objectMapper ); RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 80bf2947670f..83cdb0887c3d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -185,6 +185,7 @@ public void testIndexTaskSerde() throws Exception null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), + null, jsonMapper ), new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), @@ -247,6 +248,7 @@ public void testIndexTaskwithResourceSerde() throws Exception null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), + null, jsonMapper ), new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), @@ -469,6 +471,7 @@ public void testRealtimeIndexTaskSerde() throws Exception null, new AggregatorFactory[0], new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), + null, jsonMapper ), new RealtimeIOConfig( @@ -760,6 +763,7 @@ public void testHadoopIndexTaskSerde() throws Exception null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) ), + null, jsonMapper ), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null ), diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 3e5110d4b943..0dcfb2afc249 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -661,6 +661,7 @@ public void testIndexTask() throws Exception null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), + null, mapper ), new IndexIOConfig(new MockFirehoseFactory(false), false), @@ -718,6 +719,7 @@ public void testIndexTaskFailure() throws Exception null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) ), + null, mapper ), new IndexIOConfig(new MockExceptionalFirehoseFactory(), false), @@ -1082,6 +1084,7 @@ public void testResumeTasks() throws Exception null, ImmutableList.of(Intervals.of("2010-01-01/P2D")) ), + null, mapper ), new IndexIOConfig(new MockFirehoseFactory(false), false), @@ -1183,6 +1186,7 @@ private RealtimeIndexTask newRealtimeIndexTask() null, new AggregatorFactory[]{new LongSumAggregatorFactory("count", "rows")}, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, mapper ); RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java index 1e147f1517ff..fcd01e7f6426 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/TaskAnnouncementTest.java @@ -58,7 +58,7 @@ public void testBackwardsCompatibleSerde() throws Exception "theid", new TaskResource("rofl", 2), new FireDepartment( - new DataSchema("foo", null, new AggregatorFactory[0], null, new DefaultObjectMapper()), + new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()), new RealtimeIOConfig( new LocalFirehoseFactory(new File("lol"), "rofl", null), new PlumberSchool() { diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index eb5f88a52dc2..b5d075610a4b 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -50,6 +50,7 @@ public class DataSchema private final Map parser; private final AggregatorFactory[] aggregators; private final GranularitySpec granularitySpec; + private final TransformSpec transformSpec; private final ObjectMapper jsonMapper; @@ -59,12 +60,14 @@ public DataSchema( @JsonProperty("parser") Map parser, @JsonProperty("metricsSpec") AggregatorFactory[] aggregators, @JsonProperty("granularitySpec") GranularitySpec granularitySpec, + @JsonProperty("transformSpec") TransformSpec transformSpec, @JacksonInject ObjectMapper jsonMapper ) { this.jsonMapper = Preconditions.checkNotNull(jsonMapper, "null ObjectMapper."); this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource cannot be null. Please provide a dataSource."); this.parser = parser; + this.transformSpec = transformSpec == null ? TransformSpec.NONE : transformSpec; if (granularitySpec == null) { log.warn("No granularitySpec has been specified. Using UniformGranularitySpec as default."); @@ -108,7 +111,9 @@ public InputRowParser getParser() return null; } - final InputRowParser inputRowParser = jsonMapper.convertValue(this.parser, InputRowParser.class); + final InputRowParser inputRowParser = transformSpec.decorate( + jsonMapper.convertValue(this.parser, InputRowParser.class) + ); final Set dimensionExclusions = Sets.newHashSet(); for (AggregatorFactory aggregator : aggregators) { @@ -143,12 +148,12 @@ public InputRowParser getParser() return inputRowParser.withParseSpec( inputRowParser.getParseSpec() - .withDimensionsSpec( - dimensionsSpec - .withDimensionExclusions( - Sets.difference(dimensionExclusions, dimSet) + .withDimensionsSpec( + dimensionsSpec + .withDimensionExclusions( + Sets.difference(dimensionExclusions, dimSet) + ) ) - ) ); } else { return inputRowParser; @@ -171,9 +176,15 @@ public GranularitySpec getGranularitySpec() return granularitySpec; } + @JsonProperty + public TransformSpec getTransformSpec() + { + return transformSpec; + } + public DataSchema withGranularitySpec(GranularitySpec granularitySpec) { - return new DataSchema(dataSource, parser, aggregators, granularitySpec, jsonMapper); + return new DataSchema(dataSource, parser, aggregators, granularitySpec, transformSpec, jsonMapper); } @Override @@ -184,6 +195,7 @@ public String toString() ", parser=" + parser + ", aggregators=" + Arrays.toString(aggregators) + ", granularitySpec=" + granularitySpec + + ", transformSpec=" + transformSpec + '}'; } } diff --git a/server/src/main/java/io/druid/segment/indexing/Transform.java b/server/src/main/java/io/druid/segment/indexing/Transform.java new file mode 100644 index 000000000000..ed64b8348985 --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/Transform.java @@ -0,0 +1,95 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.druid.math.expr.Expr; +import io.druid.math.expr.ExprMacroTable; +import io.druid.math.expr.Parser; + +import java.util.Objects; + +public class Transform +{ + private final String name; + private final String expression; + private final ExprMacroTable macroTable; + + @JsonCreator + public Transform( + @JsonProperty("name") final String name, + @JsonProperty("expression") final String expression, + @JacksonInject ExprMacroTable macroTable + ) + { + this.name = Preconditions.checkNotNull(name, "name"); + this.expression = Preconditions.checkNotNull(expression, "expression"); + this.macroTable = macroTable; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getExpression() + { + return expression; + } + + public Expr toExpr() + { + return Parser.parse(expression, Preconditions.checkNotNull(macroTable, "macroTable")); + } + + @Override + public boolean equals(final Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Transform transform = (Transform) o; + return Objects.equals(name, transform.name) && + Objects.equals(expression, transform.expression); + } + + @Override + public int hashCode() + { + return Objects.hash(name, expression); + } + + @Override + public String toString() + { + return "Transform{" + + "name='" + name + '\'' + + ", expression='" + expression + '\'' + + '}'; + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/TransformSpec.java b/server/src/main/java/io/druid/segment/indexing/TransformSpec.java new file mode 100644 index 000000000000..aead6fb2202e --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/TransformSpec.java @@ -0,0 +1,124 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.StringInputRowParser; +import io.druid.java.util.common.ISE; +import io.druid.query.filter.DimFilter; + +import javax.annotation.Nullable; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +public class TransformSpec +{ + public static final TransformSpec NONE = new TransformSpec(null, null); + + private final DimFilter filter; + private final List transforms; + + @JsonCreator + public TransformSpec( + @JsonProperty("filter") final DimFilter filter, + @JsonProperty("transforms") final List transforms + ) + { + this.filter = filter; + this.transforms = transforms == null ? ImmutableList.of() : transforms; + + // Check for name collisions. + final Set seen = new HashSet<>(); + for (Transform transform : this.transforms) { + if (!seen.add(transform.getName())) { + throw new ISE("Transform name '%s' cannot be used twice", transform.getName()); + } + } + } + + @JsonProperty + @Nullable + public DimFilter getFilter() + { + return filter; + } + + @JsonProperty + public List getTransforms() + { + return transforms; + } + + public InputRowParser decorate(final InputRowParser parser) + { + if (filter == null && transforms.isEmpty()) { + return parser; + } else if (parser instanceof StringInputRowParser) { + // Hack to support the fact that some callers use special methods in StringInputRowParser, such as + // parse(String) and startFileFromBeginning. + return (InputRowParser) new TransformingStringInputRowParser( + parser.getParseSpec(), + ((StringInputRowParser) parser).getEncoding(), + this + ); + } else { + return new TransformingInputRowParser<>(parser, this); + } + } + + public Transformer toTransformer() + { + return new Transformer(this); + } + + @Override + public boolean equals(final Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final TransformSpec that = (TransformSpec) o; + return Objects.equals(filter, that.filter) && + Objects.equals(transforms, that.transforms); + } + + @Override + public int hashCode() + { + return Objects.hash(filter, transforms); + } + + @Override + public String toString() + { + return "TransformSpec{" + + "filter=" + filter + + ", transforms=" + transforms + + '}'; + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/Transformer.java b/server/src/main/java/io/druid/segment/indexing/Transformer.java new file mode 100644 index 000000000000..42213096f9cf --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/Transformer.java @@ -0,0 +1,231 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import com.google.common.base.Strings; +import io.druid.data.input.InputRow; +import io.druid.data.input.Row; +import io.druid.math.expr.Expr; +import io.druid.query.filter.ValueMatcher; +import io.druid.query.groupby.RowBasedColumnSelectorFactory; +import io.druid.segment.column.Column; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * + */ +public class Transformer +{ + private final Map transforms = new HashMap<>(); + private final ThreadLocal rowSupplierForValueMatcher = new ThreadLocal<>(); + private final ValueMatcher valueMatcher; + + Transformer(final TransformSpec transformSpec) + { + for (final Transform transform : transformSpec.getTransforms()) { + transforms.put(transform.getName(), transform.toExpr()); + } + + if (transformSpec.getFilter() != null) { + valueMatcher = transformSpec.getFilter().toFilter() + .makeMatcher( + RowBasedColumnSelectorFactory.create( + rowSupplierForValueMatcher, + null + ) + ); + } else { + valueMatcher = null; + } + } + + /** + * Transforms an input row, or returns null if the row should be filtered out. + * + * @param row the input row + */ + @Nullable + public InputRow transform(@Nullable final InputRow row) + { + if (row == null) { + return null; + } + + final InputRow transformedRow; + + if (transforms.isEmpty()) { + transformedRow = row; + } else { + transformedRow = new TransformedInputRow(row, transforms); + } + + if (valueMatcher != null) { + rowSupplierForValueMatcher.set(transformedRow); + if (!valueMatcher.matches()) { + return null; + } + } + + return transformedRow; + } + + public static class TransformedInputRow implements InputRow + { + private final InputRow row; + private final Map transforms; + + public TransformedInputRow(final InputRow row, final Map transforms) + { + this.row = row; + this.transforms = transforms; + } + + @Override + public List getDimensions() + { + return row.getDimensions(); + } + + @Override + public long getTimestampFromEpoch() + { + final Expr transform = transforms.get(Column.TIME_COLUMN_NAME); + if (transform != null) { + return transform.eval(this::getValueFromRow).asLong(); + } else { + return row.getTimestampFromEpoch(); + } + } + + @Override + public DateTime getTimestamp() + { + final Expr transform = transforms.get(Column.TIME_COLUMN_NAME); + if (transform != null) { + return new DateTime(transform.eval(this::getValueFromRow).asLong()); + } else { + return row.getTimestamp(); + } + } + + @Override + public List getDimension(final String dimension) + { + final Expr transform = transforms.get(dimension); + if (transform != null) { + // Always return single-value. Expressions don't support array/list operations yet. + final String s = transform.eval(this::getValueFromRow).asString(); + if (Strings.isNullOrEmpty(s)) { + return Collections.emptyList(); + } else { + return Collections.singletonList(s); + } + } else { + return row.getDimension(dimension); + } + } + + @Override + public Object getRaw(final String column) + { + final Expr transform = transforms.get(column); + if (transform != null) { + return transform.eval(this::getValueFromRow).value(); + } else { + return row.getRaw(column); + } + } + + @Override + public float getFloatMetric(final String metric) + { + final Expr transform = transforms.get(metric); + if (transform != null) { + return (float) transform.eval(this::getValueFromRow).asDouble(); + } else { + return row.getFloatMetric(metric); + } + } + + @Override + public long getLongMetric(final String metric) + { + final Expr transform = transforms.get(metric); + if (transform != null) { + return transform.eval(this::getValueFromRow).asLong(); + } else { + return row.getLongMetric(metric); + } + } + + @Override + public double getDoubleMetric(final String metric) + { + final Expr transform = transforms.get(metric); + if (transform != null) { + return transform.eval(this::getValueFromRow).asDouble(); + } else { + return row.getDoubleMetric(metric); + } + } + + @Override + public boolean equals(final Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final TransformedInputRow that = (TransformedInputRow) o; + return Objects.equals(row, that.row) && + Objects.equals(transforms, that.transforms); + } + + @Override + public int hashCode() + { + return Objects.hash(row, transforms); + } + + @Override + public int compareTo(final Row o) + { + return row.compareTo(o); + } + + private Object getValueFromRow(final String column) + { + if (column.equals(Column.TIME_COLUMN_NAME)) { + return row.getTimestampFromEpoch(); + } else { + return row.getRaw(column); + } + } + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java b/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java new file mode 100644 index 000000000000..d38ee9011b62 --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java @@ -0,0 +1,56 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.ParseSpec; + +public class TransformingInputRowParser implements InputRowParser +{ + private final InputRowParser parser; + private final TransformSpec transformSpec; + private final Transformer transformer; + + public TransformingInputRowParser(final InputRowParser parser, final TransformSpec transformSpec) + { + this.parser = parser; + this.transformSpec = transformSpec; + this.transformer = transformSpec.toTransformer(); + } + + @Override + public InputRow parse(final T row) + { + return transformer.transform(parser.parse(row)); + } + + @Override + public ParseSpec getParseSpec() + { + return parser.getParseSpec(); + } + + @Override + public InputRowParser withParseSpec(final ParseSpec parseSpec) + { + return new TransformingInputRowParser<>(parser.withParseSpec(parseSpec), transformSpec); + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java b/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java new file mode 100644 index 000000000000..d4a911db0e2f --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java @@ -0,0 +1,63 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.ParseSpec; +import io.druid.data.input.impl.StringInputRowParser; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; + +public class TransformingStringInputRowParser extends StringInputRowParser +{ + private final TransformSpec transformSpec; + private final Transformer transformer; + + public TransformingStringInputRowParser( + final ParseSpec parseSpec, + final String encoding, + final TransformSpec transformSpec + ) + { + super(parseSpec, encoding); + this.transformSpec = transformSpec; + this.transformer = transformSpec.toTransformer(); + } + + @Override + public InputRow parse(final ByteBuffer input) + { + return transformer.transform(super.parse(input)); + } + + @Nullable + @Override + public InputRow parse(@Nullable final String input) + { + return transformer.transform(super.parse(input)); + } + + @Override + public StringInputRowParser withParseSpec(final ParseSpec parseSpec) + { + return new TransformingStringInputRowParser(getParseSpec(), getEncoding(), transformSpec); + } +} diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java index d415875ad737..9e44b390a802 100644 --- a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -20,8 +20,10 @@ package io.druid.segment.indexing; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Charsets; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.StringInputRowParser; @@ -33,11 +35,15 @@ import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; +import io.druid.query.expression.TestExprMacroTable; +import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.TestHelper; import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; +import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Map; @@ -68,6 +74,7 @@ public void testDefaultExclusions() throws Exception new DoubleSumAggregatorFactory("metric2", "col2"), }, new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, jsonMapper ); @@ -104,6 +111,7 @@ public void testExplicitInclude() throws Exception new DoubleSumAggregatorFactory("metric2", "col2"), }, new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, jsonMapper ); @@ -113,6 +121,63 @@ public void testExplicitInclude() throws Exception ); } + @Test + public void testTransformSpec() throws Exception + { + Map parserMap = jsonMapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("time", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("time", "dimA", "dimB", "col2")), + ImmutableList.of(), + null + ), + null, + null + ), + null + ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ); + + DataSchema schema = new DataSchema( + "test", + parserMap, + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + }, + new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + new TransformSpec( + new SelectorDimFilter("dimA", "foo", null), + ImmutableList.of( + new Transform("expr", "concat(dimA,dimA)", TestExprMacroTable.INSTANCE) + ) + ), + jsonMapper + ); + + // Test hack that produces a StringInputRowParser. + final StringInputRowParser parser = (StringInputRowParser) schema.getParser(); + + final InputRow row1bb = parser.parse( + ByteBuffer.wrap("{\"time\":\"2000-01-01\",\"dimA\":\"foo\"}".getBytes(Charsets.UTF_8)) + ); + Assert.assertEquals(new DateTime("2000-01-01"), row1bb.getTimestamp()); + Assert.assertEquals("foo", row1bb.getRaw("dimA")); + Assert.assertEquals("foofoo", row1bb.getRaw("expr")); + + final InputRow row1string = parser.parse("{\"time\":\"2000-01-01\",\"dimA\":\"foo\"}"); + Assert.assertEquals(new DateTime("2000-01-01"), row1bb.getTimestamp()); + Assert.assertEquals("foo", row1bb.getRaw("dimA")); + Assert.assertEquals("foofoo", row1bb.getRaw("expr")); + + final InputRow row2 = parser.parse( + ByteBuffer.wrap("{\"time\":\"2000-01-01\",\"dimA\":\"x\"}".getBytes(Charsets.UTF_8)) + ); + Assert.assertNull(row2); + } + @Test(expected = IAE.class) public void testOverlapMetricNameAndDim() throws Exception { @@ -141,6 +206,7 @@ public void testOverlapMetricNameAndDim() throws Exception new DoubleSumAggregatorFactory("metric2", "col2"), }, new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, jsonMapper ); schema.getParser(); @@ -174,6 +240,7 @@ public void testDuplicateAggregators() throws Exception new DoubleSumAggregatorFactory("metric1", "col3"), }, new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + null, jsonMapper ); schema.getParser(); diff --git a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java new file mode 100644 index 000000000000..222acbc9daf6 --- /dev/null +++ b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java @@ -0,0 +1,152 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.query.expression.TestExprMacroTable; +import io.druid.query.filter.AndDimFilter; +import io.druid.query.filter.SelectorDimFilter; +import org.joda.time.DateTime; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class TransformSpecTest +{ + private static final MapInputRowParser PARSER = new MapInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec("t", "auto", new DateTime("2000-01-01")), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("f", "x", "y")), + null, + null + ) + ) + ); + + private static final Map ROW1 = ImmutableMap.builder() + .put("x", "foo") + .put("y", "bar") + .put("a", 2.0) + .put("b", 3L) + .build(); + + private static final Map ROW2 = ImmutableMap.builder() + .put("x", "foo") + .put("y", "baz") + .put("a", 2.0) + .put("b", 4L) + .build(); + + @Test + public void testTransforms() + { + final TransformSpec transformSpec = new TransformSpec( + null, + ImmutableList.of( + new Transform("f", "concat(x,y)", TestExprMacroTable.INSTANCE), + new Transform("g", "a + b", TestExprMacroTable.INSTANCE), + new Transform("h", "concat(f,g)", TestExprMacroTable.INSTANCE) + ) + ); + + final InputRowParser> parser = transformSpec.decorate(PARSER); + final InputRow row = parser.parse(ROW1); + + Assert.assertNotNull(row); + Assert.assertEquals(new DateTime("2000-01-01").getMillis(), row.getTimestampFromEpoch()); + Assert.assertEquals(new DateTime("2000-01-01"), row.getTimestamp()); + Assert.assertEquals(ImmutableList.of("f", "x", "y"), row.getDimensions()); + Assert.assertEquals(ImmutableList.of("foo"), row.getDimension("x")); + Assert.assertEquals(3.0, row.getDoubleMetric("b"), 0); + Assert.assertEquals("foobar", row.getRaw("f")); + Assert.assertEquals(ImmutableList.of("foobar"), row.getDimension("f")); + Assert.assertEquals(ImmutableList.of("5.0"), row.getDimension("g")); + Assert.assertEquals(ImmutableList.of(), row.getDimension("h")); + Assert.assertEquals(0L, row.getLongMetric("f")); + Assert.assertEquals(5L, row.getLongMetric("g")); + } + + @Test + public void testFilterOnTransforms() + { + final TransformSpec transformSpec = new TransformSpec( + new AndDimFilter( + ImmutableList.of( + new SelectorDimFilter("x", "foo", null), + new SelectorDimFilter("f", "foobar", null), + new SelectorDimFilter("g", "5.0", null) + ) + ), + ImmutableList.of( + new Transform("f", "concat(x,y)", TestExprMacroTable.INSTANCE), + new Transform("g", "a + b", TestExprMacroTable.INSTANCE) + ) + ); + + final InputRowParser> parser = transformSpec.decorate(PARSER); + Assert.assertNotNull(parser.parse(ROW1)); + Assert.assertNull(parser.parse(ROW2)); + } + + @Test + public void testTransformTimeFromOtherFields() + { + final TransformSpec transformSpec = new TransformSpec( + null, + ImmutableList.of( + new Transform("__time", "(a + b) * 3600000", TestExprMacroTable.INSTANCE) + ) + ); + + final InputRowParser> parser = transformSpec.decorate(PARSER); + final InputRow row = parser.parse(ROW1); + + Assert.assertNotNull(row); + Assert.assertEquals(new DateTime("1970-01-01T05:00:00Z"), row.getTimestamp()); + Assert.assertEquals(new DateTime("1970-01-01T05:00:00Z").getMillis(), row.getTimestampFromEpoch()); + } + + @Test + public void testTransformTimeFromTime() + { + final TransformSpec transformSpec = new TransformSpec( + null, + ImmutableList.of( + new Transform("__time", "__time + 3600000", TestExprMacroTable.INSTANCE) + ) + ); + + final InputRowParser> parser = transformSpec.decorate(PARSER); + final InputRow row = parser.parse(ROW1); + + Assert.assertNotNull(row); + Assert.assertEquals(new DateTime("2000-01-01T01:00:00Z"), row.getTimestamp()); + Assert.assertEquals(new DateTime("2000-01-01T01:00:00Z").getMillis(), row.getTimestampFromEpoch()); + } +} diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 91eb6fd58988..137b75b451fb 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -97,6 +97,7 @@ public void testSerde() throws Exception new CountAggregatorFactory("count") }, new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), + null, jsonMapper ), new RealtimeIOConfig( diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index f71739b50a1f..cb3489209456 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -145,6 +145,7 @@ public void setUp() throws Exception null, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), + null, jsonMapper ); schema2 = new DataSchema( @@ -152,6 +153,7 @@ public void setUp() throws Exception null, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), + null, jsonMapper ); RealtimeIOConfig ioConfig = new RealtimeIOConfig( @@ -295,6 +297,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException null, new AggregatorFactory[]{new CountAggregatorFactory("ignore")}, new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), + null, jsonMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index a18b97851ccd..4c92003639a9 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -130,6 +130,7 @@ public AppenderatorTester( new LongSumAggregatorFactory("met", "met") }, new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), + null, objectMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 52a5aa9a431e..39bc73eaa570 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -128,6 +128,7 @@ public int columnCacheSizeBytes() new LongSumAggregatorFactory("met", "met") }, new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), + null, objectMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 5f1586a0d677..5578a11f74b1 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -142,6 +142,7 @@ public void setUp() throws Exception ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), + null, jsonMapper ); @@ -161,6 +162,7 @@ public void setUp() throws Exception ), new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.YEAR, Granularities.NONE, null), + null, jsonMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index b84d154b2f07..8be568093328 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -53,6 +53,7 @@ public void testSwap() throws Exception null, new AggregatorFactory[]{new CountAggregatorFactory("rows")}, new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), + null, new DefaultObjectMapper() ); diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index 4c55781634b8..06ae9b7c5d0d 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -150,6 +150,7 @@ public void testTaskValidator() throws Exception null, new AggregatorFactory[0], new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), + null, jsonMapper ), new RealtimeIOConfig( From 089a52f9055b30987dfaff2b71a2de85092c51c8 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 2 Oct 2017 12:23:00 -0700 Subject: [PATCH 02/23] Remove forbidden api. --- .../src/main/java/io/druid/segment/indexing/Transformer.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/segment/indexing/Transformer.java b/server/src/main/java/io/druid/segment/indexing/Transformer.java index 42213096f9cf..1d90704256b8 100644 --- a/server/src/main/java/io/druid/segment/indexing/Transformer.java +++ b/server/src/main/java/io/druid/segment/indexing/Transformer.java @@ -22,6 +22,7 @@ import com.google.common.base.Strings; import io.druid.data.input.InputRow; import io.druid.data.input.Row; +import io.druid.java.util.common.DateTimes; import io.druid.math.expr.Expr; import io.druid.query.filter.ValueMatcher; import io.druid.query.groupby.RowBasedColumnSelectorFactory; @@ -126,7 +127,7 @@ public DateTime getTimestamp() { final Expr transform = transforms.get(Column.TIME_COLUMN_NAME); if (transform != null) { - return new DateTime(transform.eval(this::getValueFromRow).asLong()); + return DateTimes.utc(transform.eval(this::getValueFromRow).asLong()); } else { return row.getTimestamp(); } From 5b6b1d51a451780c8e31856a4e8c6cc50227c760 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 2 Oct 2017 13:20:32 -0700 Subject: [PATCH 03/23] Fix compile error. --- .../src/test/java/io/druid/indexer/IndexGeneratorJobTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java index f70d7f1e1e3d..8ed853bef997 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/IndexGeneratorJobTest.java @@ -496,6 +496,7 @@ public void setUp() throws Exception new UniformGranularitySpec( Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval) ), + null, mapper ), new HadoopIOConfig( From f2ec86e36511e6a7d6a429afe04f5690dec934c6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 2 Oct 2017 15:08:46 -0700 Subject: [PATCH 04/23] Fix tests. --- .../druid/segment/indexing/DataSchemaTest.java | 10 +++++----- .../segment/indexing/TransformSpecTest.java | 16 ++++++++-------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java index 9e44b390a802..e4739ba5b612 100644 --- a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -28,6 +28,7 @@ import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.IAE; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.DurationGranularity; @@ -39,7 +40,6 @@ import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.TestHelper; import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -163,14 +163,14 @@ public void testTransformSpec() throws Exception final InputRow row1bb = parser.parse( ByteBuffer.wrap("{\"time\":\"2000-01-01\",\"dimA\":\"foo\"}".getBytes(Charsets.UTF_8)) ); - Assert.assertEquals(new DateTime("2000-01-01"), row1bb.getTimestamp()); + Assert.assertEquals(DateTimes.of("2000-01-01"), row1bb.getTimestamp()); Assert.assertEquals("foo", row1bb.getRaw("dimA")); Assert.assertEquals("foofoo", row1bb.getRaw("expr")); final InputRow row1string = parser.parse("{\"time\":\"2000-01-01\",\"dimA\":\"foo\"}"); - Assert.assertEquals(new DateTime("2000-01-01"), row1bb.getTimestamp()); - Assert.assertEquals("foo", row1bb.getRaw("dimA")); - Assert.assertEquals("foofoo", row1bb.getRaw("expr")); + Assert.assertEquals(DateTimes.of("2000-01-01"), row1string.getTimestamp()); + Assert.assertEquals("foo", row1string.getRaw("dimA")); + Assert.assertEquals("foofoo", row1string.getRaw("expr")); final InputRow row2 = parser.parse( ByteBuffer.wrap("{\"time\":\"2000-01-01\",\"dimA\":\"x\"}".getBytes(Charsets.UTF_8)) diff --git a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java index 222acbc9daf6..3e299d6812ea 100644 --- a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java +++ b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java @@ -27,10 +27,10 @@ import io.druid.data.input.impl.MapInputRowParser; import io.druid.data.input.impl.TimeAndDimsParseSpec; import io.druid.data.input.impl.TimestampSpec; +import io.druid.java.util.common.DateTimes; import io.druid.query.expression.TestExprMacroTable; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.SelectorDimFilter; -import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Test; @@ -40,7 +40,7 @@ public class TransformSpecTest { private static final MapInputRowParser PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( - new TimestampSpec("t", "auto", new DateTime("2000-01-01")), + new TimestampSpec("t", "auto", DateTimes.of("2000-01-01")), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of("f", "x", "y")), null, @@ -79,8 +79,8 @@ public void testTransforms() final InputRow row = parser.parse(ROW1); Assert.assertNotNull(row); - Assert.assertEquals(new DateTime("2000-01-01").getMillis(), row.getTimestampFromEpoch()); - Assert.assertEquals(new DateTime("2000-01-01"), row.getTimestamp()); + Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), row.getTimestampFromEpoch()); + Assert.assertEquals(DateTimes.of("2000-01-01"), row.getTimestamp()); Assert.assertEquals(ImmutableList.of("f", "x", "y"), row.getDimensions()); Assert.assertEquals(ImmutableList.of("foo"), row.getDimension("x")); Assert.assertEquals(3.0, row.getDoubleMetric("b"), 0); @@ -128,8 +128,8 @@ public void testTransformTimeFromOtherFields() final InputRow row = parser.parse(ROW1); Assert.assertNotNull(row); - Assert.assertEquals(new DateTime("1970-01-01T05:00:00Z"), row.getTimestamp()); - Assert.assertEquals(new DateTime("1970-01-01T05:00:00Z").getMillis(), row.getTimestampFromEpoch()); + Assert.assertEquals(DateTimes.of("1970-01-01T05:00:00Z"), row.getTimestamp()); + Assert.assertEquals(DateTimes.of("1970-01-01T05:00:00Z").getMillis(), row.getTimestampFromEpoch()); } @Test @@ -146,7 +146,7 @@ public void testTransformTimeFromTime() final InputRow row = parser.parse(ROW1); Assert.assertNotNull(row); - Assert.assertEquals(new DateTime("2000-01-01T01:00:00Z"), row.getTimestamp()); - Assert.assertEquals(new DateTime("2000-01-01T01:00:00Z").getMillis(), row.getTimestampFromEpoch()); + Assert.assertEquals(DateTimes.of("2000-01-01T01:00:00Z"), row.getTimestamp()); + Assert.assertEquals(DateTimes.of("2000-01-01T01:00:00Z").getMillis(), row.getTimestampFromEpoch()); } } From 8bffb711bbe15ce894db2e51337df845aca8cbd7 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 11:33:17 -0700 Subject: [PATCH 05/23] Some more changes. - Add nullable annotation to Firehose.nextRow. - Add tests for index task, realtime task, kafka task, hadoop mapper, and ingestSegment firehose. --- .../input/impl/FileIteratingFirehose.java | 2 + .../druid/data/input/impl/InputRowParser.java | 7 + .../TwitterSpritzerFirehoseFactory.java | 1 + .../rocketmq/RocketMQFirehoseFactory.java | 10 +- .../rabbitmq/RabbitMQFirehoseFactory.java | 8 +- .../kafka/KafkaEightFirehoseFactory.java | 12 +- .../druid/indexing/kafka/KafkaIndexTask.java | 19 +- .../indexing/kafka/KafkaIndexTaskTest.java | 196 +++++++----- .../indexer/HadoopDruidIndexerMapper.java | 18 +- .../hadoop/DatasourceRecordReader.java | 19 +- .../druid/indexer/hadoop/SegmentInputRow.java | 13 +- .../indexer/BatchDeltaIngestionTest.java | 2 + .../indexer/HadoopDruidIndexerMapperTest.java | 215 +++++++++++++ .../druid/indexing/common/task/IndexTask.java | 2 +- .../IngestSegmentFirehoseFactory.java | 4 +- .../indexing/common/task/IndexTaskTest.java | 63 +++- .../common/task/RealtimeIndexTaskTest.java | 283 ++++++++++-------- .../IngestSegmentFirehoseFactoryTest.java | 56 +++- .../indexing/overlord/TaskLifecycleTest.java | 3 + .../io/druid/segment/indexing/DataSchema.java | 5 + .../druid/segment/indexing/TransformSpec.java | 15 + .../indexing/TransformingInputRowParser.java | 5 + .../TransformingStringInputRowParser.java | 5 + .../firehose/CombiningFirehoseFactory.java | 2 + .../EventReceiverFirehoseFactory.java | 17 +- .../firehose/FixedCountFirehoseFactory.java | 2 + .../firehose/IngestSegmentFirehose.java | 9 +- .../realtime/firehose/IrcFirehoseFactory.java | 10 +- .../realtime/firehose/PredicateFirehose.java | 2 + .../firehose/TimedShutoffFirehoseFactory.java | 2 + .../segment/realtime/plumber/Plumbers.java | 11 +- .../CombiningFirehoseFactoryTest.java | 2 + .../segment/realtime/RealtimeManagerTest.java | 1 + .../firehose/IngestSegmentFirehoseTest.java | 3 + 34 files changed, 757 insertions(+), 267 deletions(-) create mode 100644 indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java diff --git a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java index d1d0606ea49e..eb60b366877e 100644 --- a/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java +++ b/api/src/main/java/io/druid/data/input/impl/FileIteratingFirehose.java @@ -24,6 +24,7 @@ import io.druid.utils.Runnables; import org.apache.commons.io.LineIterator; +import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.util.Iterator; @@ -69,6 +70,7 @@ public boolean hasMore() return lineIterator != null && lineIterator.hasNext(); } + @Nullable @Override public InputRow nextRow() { diff --git a/api/src/main/java/io/druid/data/input/impl/InputRowParser.java b/api/src/main/java/io/druid/data/input/impl/InputRowParser.java index dcf369ca959d..b442518a8ae9 100644 --- a/api/src/main/java/io/druid/data/input/impl/InputRowParser.java +++ b/api/src/main/java/io/druid/data/input/impl/InputRowParser.java @@ -24,6 +24,8 @@ import io.druid.data.input.InputRow; import io.druid.guice.annotations.ExtensionPoint; +import javax.annotation.Nullable; + @ExtensionPoint @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = StringInputRowParser.class) @JsonSubTypes(value = { @@ -33,6 +35,11 @@ }) public interface InputRowParser { + /** + * Parse an input into an {@link InputRow}. Return null if this input should be thrown away, or throws + * {@code ParseException} if the input is unparseable. + */ + @Nullable public InputRow parse(T input); public ParseSpec getParseSpec(); diff --git a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java index 8f11561df428..db1d95b36b79 100644 --- a/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java +++ b/examples/src/main/java/io/druid/examples/twitter/TwitterSpritzerFirehoseFactory.java @@ -250,6 +250,7 @@ public boolean hasMore() } } + @Nullable @Override public InputRow nextRow() { diff --git a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java index 8075ae46defd..60c194ca1e6b 100644 --- a/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java +++ b/extensions-contrib/druid-rocketmq/src/main/java/io/druid/firehose/rocketmq/RocketMQFirehoseFactory.java @@ -34,14 +34,15 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Sets; -import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -57,7 +58,7 @@ import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.CountDownLatch; -public class RocketMQFirehoseFactory implements FirehoseFactory +public class RocketMQFirehoseFactory implements FirehoseFactory> { private static final Logger LOGGER = new Logger(RocketMQFirehoseFactory.class); @@ -139,7 +140,7 @@ private boolean hasMessagesPending() @Override public Firehose connect( - ByteBufferInputRowParser byteBufferInputRowParser, + InputRowParser byteBufferInputRowParser, File temporaryDirectory ) throws IOException, ParseException { @@ -149,7 +150,7 @@ public Firehose connect( Sets.newHashSet("feed") ); - final ByteBufferInputRowParser theParser = byteBufferInputRowParser.withParseSpec( + final InputRowParser theParser = byteBufferInputRowParser.withParseSpec( byteBufferInputRowParser.getParseSpec() .withDimensionsSpec( byteBufferInputRowParser.getParseSpec() @@ -247,6 +248,7 @@ public boolean hasMore() return hasMore; } + @Nullable @Override public InputRow nextRow() { diff --git a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java index 66c91770a2b7..bbd54aa44f96 100644 --- a/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java +++ b/extensions-contrib/rabbitmq/src/main/java/io/druid/firehose/rabbitmq/RabbitMQFirehoseFactory.java @@ -30,10 +30,10 @@ import com.rabbitmq.client.QueueingConsumer.Delivery; import com.rabbitmq.client.ShutdownListener; import com.rabbitmq.client.ShutdownSignalException; -import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; import io.druid.java.util.common.logger.Logger; import net.jodah.lyra.ConnectionOptions; import net.jodah.lyra.Connections; @@ -41,6 +41,7 @@ import net.jodah.lyra.retry.RetryPolicy; import net.jodah.lyra.util.Duration; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -100,7 +101,7 @@ * For more information on RabbitMQ high availability please see: * http://www.rabbitmq.com/ha.html. */ -public class RabbitMQFirehoseFactory implements FirehoseFactory +public class RabbitMQFirehoseFactory implements FirehoseFactory> { private static final Logger log = new Logger(RabbitMQFirehoseFactory.class); @@ -135,7 +136,7 @@ public JacksonifiedConnectionFactory getConnectionFactory() } @Override - public Firehose connect(final ByteBufferInputRowParser firehoseParser, File temporaryDirectory) throws IOException + public Firehose connect(final InputRowParser firehoseParser, File temporaryDirectory) throws IOException { ConnectionOptions lyraOptions = new ConnectionOptions(this.connectionFactory); Config lyraConfig = new Config() @@ -225,6 +226,7 @@ public boolean hasMore() return false; } + @Nullable @Override public InputRow nextRow() { diff --git a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java index ed55aeb69c00..d164571966b3 100644 --- a/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java +++ b/extensions-core/kafka-eight/src/main/java/io/druid/firehose/kafka/KafkaEightFirehoseFactory.java @@ -24,11 +24,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; - -import io.druid.data.input.ByteBufferInputRowParser; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; import io.druid.java.util.common.logger.Logger; import kafka.consumer.Consumer; import kafka.consumer.ConsumerConfig; @@ -37,6 +36,7 @@ import kafka.javaapi.consumer.ConsumerConnector; import kafka.message.InvalidMessageException; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -47,7 +47,7 @@ /** */ -public class KafkaEightFirehoseFactory implements FirehoseFactory +public class KafkaEightFirehoseFactory implements FirehoseFactory> { private static final Logger log = new Logger(KafkaEightFirehoseFactory.class); @@ -69,13 +69,14 @@ public KafkaEightFirehoseFactory( } @Override - public Firehose connect(final ByteBufferInputRowParser firehoseParser, File temporaryDirectory) throws IOException + public Firehose connect(final InputRowParser firehoseParser, File temporaryDirectory) throws IOException { Set newDimExclus = Sets.union( firehoseParser.getParseSpec().getDimensionsSpec().getDimensionExclusions(), Sets.newHashSet("feed") ); - final ByteBufferInputRowParser theParser = firehoseParser.withParseSpec( + + final InputRowParser theParser = firehoseParser.withParseSpec( firehoseParser.getParseSpec() .withDimensionsSpec( firehoseParser.getParseSpec() @@ -111,6 +112,7 @@ public boolean hasMore() return iter.hasNext(); } + @Nullable @Override public InputRow nextRow() { diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 08965c021644..0e99c810986a 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -465,17 +465,9 @@ public void run() try { final byte[] valueBytes = record.value(); - if (valueBytes == null) { - throw new ParseException("null value"); - } - - final InputRow row = Preconditions.checkNotNull(parser.parse(ByteBuffer.wrap(valueBytes)), "row"); - - final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); - final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); - - if (!beforeMinimumMessageTime && !afterMaximumMessageTime) { + final InputRow row = valueBytes == null ? null : parser.parse(ByteBuffer.wrap(valueBytes)); + if (row != null && withinMinMaxRecordTime(row)) { final String sequenceName = sequenceNames.get(record.partition()); final AppenderatorDriverAddResult addResult = driver.add( row, @@ -1215,4 +1207,11 @@ private void sendResetRequestAndWait(Map outOfRangePartiti log.makeAlert("Failed to send reset request for partitions [%s]", partitionOffsetMap.keySet()).emit(); } } + + private boolean withinMinMaxRecordTime(final InputRow row) + { + final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); + final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + return !beforeMinimumMessageTime && !afterMaximumMessageTime; + } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index a7435597461e..3dd42aaa76e9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -49,8 +49,6 @@ import io.druid.concurrent.Execs; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; -import io.druid.java.util.common.parsers.JSONPathFieldSpec; -import io.druid.java.util.common.parsers.JSONPathSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.discovery.DataNodeService; @@ -84,6 +82,9 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.parsers.JSONPathFieldSpec; +import io.druid.java.util.common.parsers.JSONPathSpec; +import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.EntryExistsException; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.metadata.SQLMetadataStorageActionHandlerFactory; @@ -103,6 +104,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.filter.SelectorDimFilter; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryEngine; import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; @@ -113,6 +115,8 @@ import io.druid.segment.TestHelper; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.loading.LocalDataSegmentPusher; @@ -186,7 +190,7 @@ public class KafkaIndexTaskTest new JSONParseSpec( new TimestampSpec("timestamp", "iso", null), new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2")), + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")), null, null ), @@ -326,7 +330,6 @@ public void testRunAfterDataInserted() throws Exception null, false ), - null, null ); @@ -350,8 +353,8 @@ public void testRunAfterDataInserted() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -370,7 +373,6 @@ public void testRunBeforeDataInserted() throws Exception null, false ), - null, null ); @@ -406,8 +408,8 @@ public void testRunBeforeDataInserted() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -426,7 +428,6 @@ public void testRunWithMinimumMessageTime() throws Exception null, false ), - null, null ); @@ -462,8 +463,8 @@ public void testRunWithMinimumMessageTime() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -482,7 +483,6 @@ public void testRunWithMaximumMessageTime() throws Exception DateTimes.of("2010"), false ), - null, null ); @@ -519,9 +519,71 @@ public void testRunWithMaximumMessageTime() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("b"), readSegmentDim1(desc2)); - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc3)); + Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc3)); + } + + @Test(timeout = 60_000L) + public void testRunWithTransformSpec() throws Exception + { + final KafkaIndexTask task = createTask( + null, + DATA_SCHEMA.withTransformSpec( + new TransformSpec( + new SelectorDimFilter("dim1", "b", null), + ImmutableList.of( + new Transform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + ) + ) + ), + new KafkaIOConfig( + "sequence0", + new KafkaPartitions(topic, ImmutableMap.of(0, 0L)), + new KafkaPartitions(topic, ImmutableMap.of(0, 5L)), + kafkaServer.consumerProperties(), + true, + false, + null, + null, + false + ), + null + ); + + final ListenableFuture future = runTask(task); + + // Wait for the task to start reading + while (task.getStatus() != KafkaIndexTask.Status.READING) { + Thread.sleep(10); + } + + // Insert data + try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { + for (ProducerRecord record : records) { + kafkaProducer.send(record).get(); + } + } + + // Wait for task to exit + Assert.assertEquals(TaskStatus.Status.SUCCESS, future.get().getStatusCode()); + + // Check metrics + Assert.assertEquals(1, task.getFireDepartmentMetrics().processed()); + Assert.assertEquals(0, task.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(4, task.getFireDepartmentMetrics().thrownAway()); + + // Check published metadata + SegmentDescriptor desc1 = SD(task, "2009/P1D", 0); + Assert.assertEquals(ImmutableSet.of(desc1), publishedDescriptors()); + Assert.assertEquals( + new KafkaDataSourceMetadata(new KafkaPartitions(topic, ImmutableMap.of(0, 5L))), + metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) + ); + + // Check segments in deep storage + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", desc1)); } @Test(timeout = 60_000L) @@ -547,7 +609,6 @@ public void testRunOnNothing() throws Exception null, false ), - null, null ); @@ -590,7 +651,6 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception null, false ), - null, null ); @@ -614,8 +674,8 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -644,7 +704,6 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio null, false ), - null, null ); @@ -668,8 +727,8 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -697,7 +756,6 @@ public void testReportParseExceptions() throws Exception null, false ), - null, null ); @@ -732,7 +790,6 @@ public void testRunReplicas() throws Exception null, false ), - null, null ); final KafkaIndexTask task2 = createTask( @@ -748,7 +805,6 @@ public void testRunReplicas() throws Exception null, false ), - null, null ); @@ -784,8 +840,8 @@ public void testRunReplicas() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -804,7 +860,6 @@ public void testRunConflicting() throws Exception null, false ), - null, null ); final KafkaIndexTask task2 = createTask( @@ -820,7 +875,6 @@ public void testRunConflicting() throws Exception null, false ), - null, null ); @@ -844,8 +898,8 @@ public void testRunConflicting() throws Exception Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); - Assert.assertEquals(2, task2.getFireDepartmentMetrics().unparseable()); - Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().thrownAway()); // Check published segments & metadata, should all be from the first task SegmentDescriptor desc1 = SD(task1, "2010/P1D", 0); @@ -857,8 +911,8 @@ public void testRunConflicting() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -877,7 +931,6 @@ public void testRunConflictingWithoutTransactions() throws Exception null, false ), - null, null ); final KafkaIndexTask task2 = createTask( @@ -893,7 +946,6 @@ public void testRunConflictingWithoutTransactions() throws Exception null, false ), - null, null ); @@ -923,8 +975,8 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(0, task1.getFireDepartmentMetrics().unparseable()); Assert.assertEquals(0, task1.getFireDepartmentMetrics().thrownAway()); Assert.assertEquals(3, task2.getFireDepartmentMetrics().processed()); - Assert.assertEquals(2, task2.getFireDepartmentMetrics().unparseable()); - Assert.assertEquals(0, task2.getFireDepartmentMetrics().thrownAway()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().unparseable()); + Assert.assertEquals(1, task2.getFireDepartmentMetrics().thrownAway()); // Check published segments & metadata SegmentDescriptor desc3 = SD(task2, "2011/P1D", 1); @@ -933,10 +985,10 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource())); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3)); - Assert.assertEquals(ImmutableList.of("f"), readSegmentDim1(desc4)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3)); + Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", desc4)); } @Test(timeout = 60_000L) @@ -955,7 +1007,6 @@ public void testRunOneTaskTwoPartitions() throws Exception null, false ), - null, null ); @@ -989,13 +1040,13 @@ public void testRunOneTaskTwoPartitions() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc4)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc4)); // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically Assert.assertEquals( ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), - ImmutableSet.of(readSegmentDim1(desc2), readSegmentDim1(desc3)) + ImmutableSet.of(readSegmentColumn("dim1", desc2), readSegmentColumn("dim1", desc3)) ); } @@ -1015,7 +1066,6 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, false ), - null, null ); final KafkaIndexTask task2 = createTask( @@ -1031,7 +1081,6 @@ public void testRunTwoTasksTwoPartitions() throws Exception null, false ), - null, null ); @@ -1068,9 +1117,9 @@ public void testRunTwoTasksTwoPartitions() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); - Assert.assertEquals(ImmutableList.of("g"), readSegmentDim1(desc3)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", desc3)); } @Test(timeout = 60_000L) @@ -1089,7 +1138,6 @@ public void testRestore() throws Exception null, false ), - null, null ); @@ -1126,7 +1174,6 @@ public void testRestore() throws Exception null, false ), - null, null ); @@ -1160,8 +1207,8 @@ public void testRestore() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -1180,7 +1227,6 @@ public void testRunWithPauseAndResume() throws Exception null, false ), - null, null ); @@ -1246,8 +1292,8 @@ public void testRunWithPauseAndResume() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc2)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc2)); } @Test(timeout = 60_000L) @@ -1266,7 +1312,6 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception null, false ), - null, null ); @@ -1334,9 +1379,9 @@ public void testRunAndPauseAfterReadWithModifiedEndOffsets() throws Exception ); // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("b"), readSegmentDim1(desc1)); - Assert.assertEquals(ImmutableList.of("c"), readSegmentDim1(desc2)); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentDim1(desc3)); + Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", desc1)); + Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", desc2)); + Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", desc3)); } @Test(timeout = 30_000L) @@ -1355,7 +1400,6 @@ public void testRunWithOffsetOutOfRangeExceptionAndPause() throws Exception null, false ), - null, null ); @@ -1395,7 +1439,6 @@ public void testRunWithOffsetOutOfRangeExceptionAndNextOffsetGreaterThanLeastAva null, false ), - null, true ); @@ -1465,13 +1508,22 @@ public boolean apply(TaskLock lock) private KafkaIndexTask createTask( final String taskId, final KafkaIOConfig ioConfig, - final Integer maxRowsPerSegment, + final Boolean resetOffsetAutomatically + ) + { + return createTask(taskId, DATA_SCHEMA, ioConfig, resetOffsetAutomatically); + } + + private KafkaIndexTask createTask( + final String taskId, + final DataSchema dataSchema, + final KafkaIOConfig ioConfig, final Boolean resetOffsetAutomatically ) { final KafkaTuningConfig tuningConfig = new KafkaTuningConfig( 1000, - maxRowsPerSegment, + null, new Period("P1Y"), null, null, @@ -1484,7 +1536,7 @@ private KafkaIndexTask createTask( final KafkaIndexTask task = new KafkaIndexTask( taskId, null, - DATA_SCHEMA, + dataSchema, tuningConfig, ioConfig, null, @@ -1686,7 +1738,7 @@ private File getSegmentDirectory() return new File(directory, "segments"); } - private List readSegmentDim1(final SegmentDescriptor descriptor) throws IOException + private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException { File indexZip = new File( StringUtils.format( @@ -1718,11 +1770,11 @@ private List readSegmentDim1(final SegmentDescriptor descriptor) throws ); IndexIO indexIO = new TestUtils().getTestIndexIO(); QueryableIndex index = indexIO.loadIndex(outputLocation); - DictionaryEncodedColumn dim1 = index.getColumn("dim1").getDictionaryEncoding(); + DictionaryEncodedColumn theColumn = index.getColumn(column).getDictionaryEncoding(); List values = Lists.newArrayList(); - for (int i = 0; i < dim1.length(); i++) { - int id = dim1.getSingleValueRow(i); - String value = dim1.lookupName(id); + for (int i = 0; i < theColumn.length(); i++) { + int id = theColumn.getSingleValueRow(i); + String value = theColumn.lookupName(id); values.add(value); } return values; diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java index d50549c1ee9a..78d9e31d3fcf 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java @@ -30,6 +30,7 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Mapper; +import javax.annotation.Nullable; import java.io.IOException; public abstract class HadoopDruidIndexerMapper extends Mapper @@ -75,10 +76,19 @@ protected void map( if (reportParseExceptions) { throw e; } - log.debug(e, "Ignoring invalid row [%s] due to parsing error", value.toString()); + if (log.isDebugEnabled()) { + log.debug(e, "Ignoring invalid row [%s] due to parsing error", value.toString()); + } context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER).increment(1); return; // we're ignoring this invalid row + } + if (inputRow == null) { + // Throw away null rows from the parser. + if (log.isDebugEnabled()) { + log.debug("Throwing away row [%s]", value.toString()); + } + return; } if (!granularitySpec.bucketIntervals().isPresent() @@ -92,7 +102,8 @@ protected void map( } } - public final static InputRow parseInputRow(Object value, InputRowParser parser) + @Nullable + public static InputRow parseInputRow(Object value, InputRowParser parser) { if (parser instanceof StringInputRowParser && value instanceof Text) { //Note: This is to ensure backward compatibility with 0.7.0 and before @@ -101,6 +112,9 @@ public final static InputRow parseInputRow(Object value, InputRowParser parser) return ((StringInputRowParser) parser).parse(value.toString()); } else if (value instanceof InputRow) { return (InputRow) value; + } else if (value == null) { + // Pass through nulls so they get thrown away. + return null; } else { return parser.parse(value); } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java index be6eb64a268a..94129d72030e 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java @@ -27,8 +27,7 @@ import com.google.common.io.Closeables; import com.google.common.io.Files; import io.druid.data.input.InputRow; -import io.druid.data.input.MapBasedInputRow; -import io.druid.data.input.MapBasedRow; +import io.druid.data.input.Row; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.JobHelper; import io.druid.java.util.common.ISE; @@ -57,7 +56,7 @@ public class DatasourceRecordReader extends RecordReader private IngestSegmentFirehose firehose; private int rowNum; - private MapBasedRow currRow; + private Row currRow; private List indexes = Lists.newArrayList(); private List tmpSegmentDirs = Lists.newArrayList(); @@ -106,20 +105,22 @@ public WindowedStorageAdapter apply(WindowedDataSegment segment) } ); + final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); + firehose = new IngestSegmentFirehose( adapters, + config.getSchema().getDataSchema().getTransformSpec(), spec.getDimensions(), spec.getMetrics(), spec.getFilter() ); - } @Override public boolean nextKeyValue() throws IOException, InterruptedException { if (firehose.hasMore()) { - currRow = (MapBasedRow) firehose.nextRow(); + currRow = firehose.nextRow(); rowNum++; return true; } else { @@ -136,13 +137,7 @@ public NullWritable getCurrentKey() throws IOException, InterruptedException @Override public InputRow getCurrentValue() throws IOException, InterruptedException { - return new SegmentInputRow( - new MapBasedInputRow( - currRow.getTimestamp(), - spec.getDimensions(), - currRow.getEvent() - ) - ); + return currRow == null ? null : new SegmentInputRow(currRow, spec.getDimensions()); } @Override diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java index 93f6d59e784b..8a8a811dc08a 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/SegmentInputRow.java @@ -33,17 +33,19 @@ */ public class SegmentInputRow implements InputRow { - private final InputRow delegate; + private final Row delegate; + private final List dimensions; - public SegmentInputRow(InputRow delegate) + public SegmentInputRow(Row delegate, List dimensions) { this.delegate = delegate; + this.dimensions = dimensions; } @Override public List getDimensions() { - return delegate.getDimensions(); + return dimensions; } @Override @@ -94,11 +96,6 @@ public int compareTo(Row row) return delegate.compareTo(row); } - public InputRow getDelegate() - { - return delegate; - } - @Override public String toString() { diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java index e751e31282a9..efc5771a2d26 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/BatchDeltaIngestionTest.java @@ -47,6 +47,7 @@ import io.druid.segment.QueryableIndexStorageAdapter; import io.druid.segment.StorageAdapter; import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.LocalDataSegmentPuller; import io.druid.segment.realtime.firehose.IngestSegmentFirehose; @@ -322,6 +323,7 @@ private void testIngestion( Firehose firehose = new IngestSegmentFirehose( ImmutableList.of(new WindowedStorageAdapter(adapter, windowedDataSegment.getInterval())), + TransformSpec.NONE, ImmutableList.of("host"), ImmutableList.of("visited_sum", "unique_hosts"), null diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java new file mode 100644 index 000000000000..34ff0421d618 --- /dev/null +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -0,0 +1,215 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.indexer; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; +import io.druid.data.input.InputRow; +import io.druid.data.input.impl.DimensionsSpec; +import io.druid.data.input.impl.JSONParseSpec; +import io.druid.data.input.impl.TimestampSpec; +import io.druid.indexer.path.StaticPathSpec; +import io.druid.java.util.common.granularity.Granularities; +import io.druid.java.util.common.jackson.JacksonUtils; +import io.druid.java.util.common.parsers.JSONPathSpec; +import io.druid.math.expr.ExprMacroTable; +import io.druid.query.aggregation.AggregatorFactory; +import io.druid.query.aggregation.CountAggregatorFactory; +import io.druid.query.filter.SelectorDimFilter; +import io.druid.segment.TestHelper; +import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.TransformSpec; +import io.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Mapper; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class HadoopDruidIndexerMapperTest +{ + private static final ObjectMapper JSON_MAPPER = TestHelper.getJsonMapper(); + private static final DataSchema DATA_SCHEMA = new DataSchema( + "test_ds", + JSON_MAPPER.convertValue( + new HadoopyStringInputRowParser( + new JSONParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ), + new AggregatorFactory[]{new CountAggregatorFactory("rows")}, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + JSON_MAPPER + ); + + private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( + JSON_MAPPER.convertValue( + new StaticPathSpec("dummyPath", null), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ), + null, + "dummyOutputPath" + ); + + private static final HadoopTuningConfig TUNING_CONFIG = HadoopTuningConfig + .makeDefaultTuningConfig() + .withWorkingPath("dummyWorkingPath"); + + @Test + public void testHadoopyStringParser() throws Exception + { + final HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( + new HadoopIngestionSpec(DATA_SCHEMA, IO_CONFIG, TUNING_CONFIG) + ); + + final MyMapper mapper = new MyMapper(); + final Configuration hadoopConfig = new Configuration(); + hadoopConfig.set( + HadoopDruidIndexerConfig.CONFIG_PROPERTY, + HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(config) + ); + final Mapper.Context mapContext = EasyMock.mock(Mapper.Context.class); + EasyMock.expect(mapContext.getConfiguration()).andReturn(hadoopConfig).once(); + EasyMock.replay(mapContext); + mapper.setup(mapContext); + final List> rows = ImmutableList.of( + ImmutableMap.of("t", "2000-01-01T00:00:00.000Z", "dim1", "x", "m1", 1.0), + ImmutableMap.of("t", "2000-01-01T00:00:00.000Z", "dim2", "y", "m1", 1.0) + ); + for (Map row : rows) { + mapper.map(NullWritable.get(), new Text(JSON_MAPPER.writeValueAsString(row)), mapContext); + } + assertRowListEquals(rows, mapper.getRows()); + } + + @Test + public void testHadoopyStringParserWithTransformSpec() throws Exception + { + final HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( + new HadoopIngestionSpec( + DATA_SCHEMA.withTransformSpec( + new TransformSpec( + new SelectorDimFilter("dim1", "foo", null), + ImmutableList.of( + new Transform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + ) + ) + ), + IO_CONFIG, + TUNING_CONFIG + ) + ); + + final MyMapper mapper = new MyMapper(); + final Configuration hadoopConfig = new Configuration(); + hadoopConfig.set( + HadoopDruidIndexerConfig.CONFIG_PROPERTY, + HadoopDruidIndexerConfig.JSON_MAPPER.writeValueAsString(config) + ); + final Mapper.Context mapContext = EasyMock.mock(Mapper.Context.class); + EasyMock.expect(mapContext.getConfiguration()).andReturn(hadoopConfig).once(); + EasyMock.replay(mapContext); + mapper.setup(mapContext); + final List> rows = ImmutableList.of( + ImmutableMap.of("t", "2000-01-01T00:00:00.000Z", "dim1", "foo", "dim2", "x", "m1", 1.0), + ImmutableMap.of("t", "2000-01-01T00:00:00.000Z", "dim1", "bar", "dim2", "y", "m1", 1.0), + ImmutableMap.of("t", "2000-01-01T00:00:00.000Z", "dim1", "foo", "dim2", "z", "m1", 1.0) + ); + for (Map row : rows) { + mapper.map(NullWritable.get(), new Text(JSON_MAPPER.writeValueAsString(row)), mapContext); + } + assertRowListEquals( + ImmutableList.of( + ImmutableMap.of("t", "2000-01-01T00:00:00.000Z", "dim1", "foo", "dim1t", "foofoo", "dim2", "x", "m1", 1.0), + ImmutableMap.of("t", "2000-01-01T00:00:00.000Z", "dim1", "foo", "dim1t", "foofoo", "dim2", "z", "m1", 1.0) + ), + mapper.getRows() + ); + } + + private static void assertRowListEquals(final List> expected, final List actual) + { + Assert.assertEquals( + expected, + actual.stream().map(HadoopDruidIndexerMapperTest::rowToMap).collect(Collectors.toList()) + ); + } + + private static Map rowToMap(final InputRow row) + { + // Normalize input row for the purposes of testing. + final ImmutableMap.Builder builder = ImmutableMap.builder() + .put("t", row.getTimestamp().toString()); + + for (String dim : row.getDimensions()) { + final Object val = row.getRaw(dim); + if (val != null) { + builder.put(dim, val); + } + } + + // other, non-dimension fields are not self describing so much be specified individually + builder.put("m1", row.getRaw("m1")); + return builder.build(); + } + + public static class MyMapper extends HadoopDruidIndexerMapper + { + private final List rows = new ArrayList<>(); + + @Override + protected void innerMap( + final InputRow inputRow, + final Object value, + final Context context, + final boolean reportParseExceptions + ) throws IOException, InterruptedException + { + rows.add(inputRow); + } + + public List getRows() + { + return rows; + } + } +} diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 3aa0e5c2444b..228d1a8bd719 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -630,6 +630,7 @@ dataSchema, new RealtimeIOConfig(null, null, null), null final InputRow inputRow = firehose.nextRow(); if (inputRow == null) { + fireDepartmentMetrics.incrementThrownAway(); continue; } @@ -639,7 +640,6 @@ dataSchema, new RealtimeIOConfig(null, null, null), null continue; } - final String sequenceName; if (isGuaranteedRollup) { diff --git a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 231c8b19ebdc..18427125f9e3 100644 --- a/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -42,6 +42,7 @@ import io.druid.query.filter.DimFilter; import io.druid.segment.IndexIO; import io.druid.segment.QueryableIndexStorageAdapter; +import io.druid.segment.indexing.TransformSpec; import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.realtime.firehose.IngestSegmentFirehose; import io.druid.segment.realtime.firehose.WindowedStorageAdapter; @@ -281,7 +282,8 @@ public WindowedStorageAdapter apply(final PartitionChunk input) ) ); - return new IngestSegmentFirehose(adapters, dims, metricsList, dimFilter); + final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser); + return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter); } catch (IOException | SegmentLoadingException e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index f59a97d70426..13f055967e03 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -20,6 +20,7 @@ package io.druid.indexing.common.task; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.io.Files; @@ -46,12 +47,16 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.parsers.ParseException; +import io.druid.math.expr.ExprMacroTable; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -207,6 +212,50 @@ public void testForceExtendableShardSpecs() throws Exception Assert.assertEquals(1, segments.get(1).getShardSpec().getPartitionNum()); } + @Test + public void testTransformSpec() throws Exception + { + File tmpDir = temporaryFolder.newFolder(); + + File tmpFile = File.createTempFile("druid", "index", tmpDir); + + try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) { + writer.write("2014-01-01T00:00:10Z,a,1\n"); + writer.write("2014-01-01T01:00:20Z,b,1\n"); + writer.write("2014-01-01T02:00:30Z,c,1\n"); + } + + IndexTask indexTask = new IndexTask( + null, + null, + createIngestionSpec( + tmpDir, + null, + new TransformSpec( + new SelectorDimFilter("dim", "b", null), + ImmutableList.of( + new Transform("dimt", "concat(dim,dim)", ExprMacroTable.nil()) + ) + ), + null, + createTuningConfig(2, null, true, false), + false + ), + null + ); + + Assert.assertEquals(indexTask.getId(), indexTask.getGroupId()); + + final List segments = runTask(indexTask); + + Assert.assertEquals(1, segments.size()); + + Assert.assertEquals("test", segments.get(0).getDataSource()); + Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(0).getInterval()); + Assert.assertEquals(NumberedShardSpec.class, segments.get(0).getShardSpec().getClass()); + Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); + } + @Test public void testWithArbitraryGranularity() throws Exception { @@ -989,6 +1038,18 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( IndexTuningConfig tuningConfig, boolean appendToExisting ) + { + return createIngestionSpec(baseDir, parseSpec, TransformSpec.NONE, granularitySpec, tuningConfig, appendToExisting); + } + + private IndexTask.IndexIngestionSpec createIngestionSpec( + File baseDir, + ParseSpec parseSpec, + TransformSpec transformSpec, + GranularitySpec granularitySpec, + IndexTuningConfig tuningConfig, + boolean appendToExisting + ) { return new IndexTask.IndexIngestionSpec( new DataSchema( @@ -1008,7 +1069,7 @@ private IndexTask.IndexIngestionSpec createIngestionSpec( Granularities.MINUTE, Arrays.asList(Intervals.of("2014/2015")) ), - null, + transformSpec, jsonMapper ), new IndexTask.IndexIOConfig( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index c9f51ac96083..5b323db6f30c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -43,8 +43,11 @@ import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; +import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.TimeAndDimsParseSpec; +import io.druid.data.input.impl.TimestampSpec; import io.druid.discovery.DataNodeService; import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.LookupNodeService; @@ -74,7 +77,9 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.jackson.JacksonUtils; import io.druid.java.util.common.parsers.ParseException; +import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.EntryExistsException; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; @@ -91,14 +96,19 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.filter.DimFilter; +import io.druid.query.filter.SelectorDimFilter; import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryEngine; import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; import io.druid.query.timeseries.TimeseriesResultValue; +import io.druid.segment.TestHelper; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; +import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderLocalCacheManager; @@ -130,6 +140,7 @@ import java.nio.file.Files; import java.util.ArrayList; import java.util.Arrays; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.concurrent.Callable; @@ -155,10 +166,16 @@ public class RealtimeIndexTaskTest private static class TestFirehose implements Firehose { - private final List queue = Lists.newLinkedList(); + private final InputRowParser> parser; + private final List> queue = new LinkedList<>(); private boolean closed = false; - public void addRows(List rows) + public TestFirehose(final InputRowParser> parser) + { + this.parser = parser; + } + + public void addRows(List> rows) { synchronized (this) { queue.addAll(rows); @@ -187,8 +204,8 @@ public boolean hasMore() public InputRow nextRow() { synchronized (this) { - final InputRow row = queue.remove(0); - if (row != null && row.getDimensions().contains(FAIL_DIM)) { + final InputRow row = parser.parse(queue.remove(0)); + if (row != null && row.getRaw(FAIL_DIM) != null) { throw new ParseException(FAIL_DIM); } return row; @@ -198,14 +215,7 @@ public InputRow nextRow() @Override public Runnable commit() { - return new Runnable() - { - @Override - public void run() - { - // do nothing - } - }; + return () -> {}; } @Override @@ -218,16 +228,17 @@ public void close() throws IOException } } - private static class TestFirehoseFactory implements FirehoseFactory + private static class TestFirehoseFactory implements FirehoseFactory { public TestFirehoseFactory() { } @Override + @SuppressWarnings("unchecked") public Firehose connect(InputRowParser parser, File temporaryDirectory) throws IOException, ParseException { - return new TestFirehose(); + return new TestFirehose(parser); } } @@ -277,7 +288,7 @@ public void testDefaultResource() throws Exception public void testHandoffTimeout() throws Exception { final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); - final RealtimeIndexTask task = makeRealtimeTask(null, true, 100L); + final RealtimeIndexTask task = makeRealtimeTask(null, TransformSpec.NONE, true, 100L); final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); final ListenableFuture statusFuture = runTask(task, taskToolbox); @@ -289,12 +300,8 @@ public void testHandoffTimeout() throws Exception final TestFirehose firehose = (TestFirehose) task.getFirehose(); firehose.addRows( - ImmutableList.of( - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "1") - ) + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1") ) ); @@ -331,22 +338,10 @@ public void testBasics() throws Exception final TestFirehose firehose = (TestFirehose) task.getFirehose(); firehose.addRows( - ImmutableList.of( - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "1") - ), - new MapBasedInputRow( - now.minus(new Period("P1D")), - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", 2.0) - ), - new MapBasedInputRow( - now, - ImmutableList.of("dim2"), - ImmutableMap.of("dim2", "bar", "met1", 2.0) - ) + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) ) ); @@ -366,8 +361,79 @@ public void testBasics() throws Exception Assert.assertEquals(0, task.getMetrics().unparseable()); // Do some queries. - Assert.assertEquals(2, sumMetric(task, "rows")); - Assert.assertEquals(3, sumMetric(task, "met1")); + Assert.assertEquals(2, sumMetric(task, null, "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); + + // Simulate handoff. + for (Map.Entry> entry : handOffCallbacks.entrySet()) { + final Pair executorRunnablePair = entry.getValue(); + Assert.assertEquals( + new SegmentDescriptor( + publishedSegment.getInterval(), + publishedSegment.getVersion(), + publishedSegment.getShardSpec().getPartitionNum() + ), + entry.getKey() + ); + executorRunnablePair.lhs.execute(executorRunnablePair.rhs); + } + handOffCallbacks.clear(); + + // Wait for the task to finish. + final TaskStatus taskStatus = statusFuture.get(); + Assert.assertEquals(TaskStatus.Status.SUCCESS, taskStatus.getStatusCode()); + } + + @Test(timeout = 60_000L) + public void testTransformSpec() throws Exception + { + final TestIndexerMetadataStorageCoordinator mdc = new TestIndexerMetadataStorageCoordinator(); + final TransformSpec transformSpec = new TransformSpec( + new SelectorDimFilter("dim1", "foo", null), + ImmutableList.of( + new Transform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + ) + ); + final RealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); + final TaskToolbox taskToolbox = makeToolbox(task, mdc, tempFolder.newFolder()); + final ListenableFuture statusFuture = runTask(task, taskToolbox); + final DataSegment publishedSegment; + + // Wait for firehose to show up, it starts off null. + while (task.getFirehose() == null) { + Thread.sleep(50); + } + + final TestFirehose firehose = (TestFirehose) task.getFirehose(); + + firehose.addRows( + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", 2.0), + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) + ) + ); + + // Stop the firehose, this will drain out existing events. + firehose.close(); + + // Wait for publish. + while (mdc.getPublished().isEmpty()) { + Thread.sleep(50); + } + + publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); + + // Check metrics. + Assert.assertEquals(1, task.getMetrics().processed()); + Assert.assertEquals(2, task.getMetrics().thrownAway()); + Assert.assertEquals(0, task.getMetrics().unparseable()); + + // Do some queries. + Assert.assertEquals(1, sumMetric(task, null, "rows")); + Assert.assertEquals(1, sumMetric(task, new SelectorDimFilter("dim1t", "foofoo", null), "rows")); + Assert.assertEquals(0, sumMetric(task, new SelectorDimFilter("dim1t", "barbar", null), "rows")); + Assert.assertEquals(1, sumMetric(task, null, "met1")); // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { @@ -405,27 +471,11 @@ public void testReportParseExceptionsOnBadMetric() throws Exception final TestFirehose firehose = (TestFirehose) task.getFirehose(); firehose.addRows( - ImmutableList.of( - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "1") - ), - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "foo") - ), - new MapBasedInputRow( - now.minus(new Period("P1D")), - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "foo") - ), - new MapBasedInputRow( - now, - ImmutableList.of("dim2"), - ImmutableMap.of("dim2", "bar", "met1", 2.0) - ) + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), + ImmutableMap.of("t", now.minus(new Period("P1D")).getMillis(), "dim1", "foo", "met1", "foo"), + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) ) ); @@ -472,39 +522,24 @@ public void testNoReportParseExceptions() throws Exception final TestFirehose firehose = (TestFirehose) task.getFirehose(); firehose.addRows( - Arrays.asList( + Arrays.asList( // Good row- will be processed. - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "1") - ), - // Null row- will be unparseable. + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "1"), + + // Null row- will be thrown away. null, + // Bad metric- will count as processed, but that particular metric won't update. - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", "foo") - ), + ImmutableMap.of("t", now.getMillis(), "dim1", "foo", "met1", "foo"), + // Bad row- will be unparseable. - new MapBasedInputRow( - now, - ImmutableList.of("dim1", FAIL_DIM), - ImmutableMap.of("dim1", "foo", "met1", 2.0) - ), + ImmutableMap.of("dim1", "foo", "met1", 2.0, FAIL_DIM, "x"), + // Old row- will be thrownAway. - new MapBasedInputRow( - now.minus(new Period("P1D")), - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo", "met1", 2.0) - ), + ImmutableMap.of("t", now.minus(Period.days(1)).getMillis(), "dim1", "foo", "met1", 2.0), + // Good row- will be processed. - new MapBasedInputRow( - now, - ImmutableList.of("dim2"), - ImmutableMap.of("dim2", "bar", "met1", 2.0) - ) + ImmutableMap.of("t", now.getMillis(), "dim2", "bar", "met1", 2.0) ) ); @@ -524,8 +559,8 @@ public void testNoReportParseExceptions() throws Exception Assert.assertEquals(2, task.getMetrics().unparseable()); // Do some queries. - Assert.assertEquals(3, sumMetric(task, "rows")); - Assert.assertEquals(3, sumMetric(task, "met1")); + Assert.assertEquals(3, sumMetric(task, null, "rows")); + Assert.assertEquals(3, sumMetric(task, null, "met1")); // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { @@ -568,12 +603,8 @@ public void testRestore() throws Exception final TestFirehose firehose = (TestFirehose) task1.getFirehose(); firehose.addRows( - ImmutableList.of( - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo") - ) + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo") ) ); @@ -601,17 +632,13 @@ public void testRestore() throws Exception } // Do a query, at this point the previous data should be loaded. - Assert.assertEquals(1, sumMetric(task2, "rows")); + Assert.assertEquals(1, sumMetric(task2, null, "rows")); final TestFirehose firehose = (TestFirehose) task2.getFirehose(); firehose.addRows( - ImmutableList.of( - new MapBasedInputRow( - now, - ImmutableList.of("dim2"), - ImmutableMap.of("dim2", "bar") - ) + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim2", "bar") ) ); @@ -626,7 +653,7 @@ public void testRestore() throws Exception publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); // Do a query. - Assert.assertEquals(2, sumMetric(task2, "rows")); + Assert.assertEquals(2, sumMetric(task2, null, "rows")); // Simulate handoff. for (Map.Entry> entry : handOffCallbacks.entrySet()) { @@ -671,12 +698,8 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception final TestFirehose firehose = (TestFirehose) task1.getFirehose(); firehose.addRows( - ImmutableList.of( - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo") - ) + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo") ) ); @@ -691,7 +714,7 @@ public void testRestoreAfterHandoffAttemptDuringShutdown() throws Exception publishedSegment = Iterables.getOnlyElement(mdc.getPublished()); // Do a query. - Assert.assertEquals(1, sumMetric(task1, "rows")); + Assert.assertEquals(1, sumMetric(task1, null, "rows")); // Trigger graceful shutdown. task1.stopGracefully(); @@ -768,12 +791,8 @@ public void testRestoreCorruptData() throws Exception final TestFirehose firehose = (TestFirehose) task1.getFirehose(); firehose.addRows( - ImmutableList.of( - new MapBasedInputRow( - now, - ImmutableList.of("dim1"), - ImmutableMap.of("dim1", "foo") - ) + ImmutableList.of( + ImmutableMap.of("t", now.getMillis(), "dim1", "foo") ) ); @@ -863,23 +882,40 @@ public TaskStatus call() throws Exception private RealtimeIndexTask makeRealtimeTask(final String taskId) { - return makeRealtimeTask(taskId, true, 0); + return makeRealtimeTask(taskId, TransformSpec.NONE, true, 0); } private RealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportParseExceptions) { - return makeRealtimeTask(taskId, reportParseExceptions, 0); + return makeRealtimeTask(taskId, TransformSpec.NONE, reportParseExceptions, 0); } - private RealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportParseExceptions, long handoffTimeout) + private RealtimeIndexTask makeRealtimeTask( + final String taskId, + final TransformSpec transformSpec, + final boolean reportParseExceptions, + final long handoffTimeout + ) { ObjectMapper objectMapper = new DefaultObjectMapper(); DataSchema dataSchema = new DataSchema( "test_ds", - null, + TestHelper.getJsonMapper().convertValue( + new MapInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2", "dim1t")), + null, + null + ) + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ), new AggregatorFactory[]{new CountAggregatorFactory("rows"), new LongSumAggregatorFactory("met1", "met1")}, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, + transformSpec, objectMapper ); RealtimeIOConfig realtimeIOConfig = new RealtimeIOConfig( @@ -1066,11 +1102,12 @@ public List getLocations() return toolboxFactory.build(task); } - public long sumMetric(final Task task, final String metric) throws Exception + public long sumMetric(final Task task, final DimFilter filter, final String metric) throws Exception { // Do a query. TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test_ds") + .filters(filter) .aggregators( ImmutableList.of( new LongSumAggregatorFactory(metric, metric) diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 40ab84887b3e..e97b4009bac3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -33,14 +33,13 @@ import com.google.inject.Guice; import com.google.inject.Module; import com.metamx.emitter.service.ServiceEmitter; -import io.druid.java.util.common.Intervals; -import io.druid.java.util.common.JodaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.InputRowParser; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; import io.druid.data.input.impl.SpatialDimensionSchema; +import io.druid.data.input.impl.TimeAndDimsParseSpec; import io.druid.data.input.impl.TimestampSpec; import io.druid.guice.GuiceAnnotationIntrospector; import io.druid.guice.GuiceInjectableValues; @@ -56,8 +55,11 @@ import io.druid.indexing.overlord.TaskLockbox; import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.java.util.common.IOE; +import io.druid.java.util.common.Intervals; +import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; +import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.IndexerSQLMetadataStorageCoordinator; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -65,8 +67,12 @@ import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; +import io.druid.segment.TestHelper; +import io.druid.segment.column.Column; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; +import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.TransformSpec; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; import io.druid.segment.loading.DataSegmentMover; @@ -115,7 +121,7 @@ public class IngestSegmentFirehoseFactoryTest static { TestUtils testUtils = new TestUtils(); - MAPPER = setupInjectablesInObjectMapper(testUtils.getTestObjectMapper()); + MAPPER = setupInjectablesInObjectMapper(TestHelper.getJsonMapper()); INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9(); INDEX_IO = testUtils.getTestIndexIO(); } @@ -420,15 +426,13 @@ public IngestSegmentFirehoseFactoryTest( private final InputRowParser rowParser; private static final InputRowParser> ROW_PARSER = new MapInputRowParser( - new JSONParseSpec( + new TimeAndDimsParseSpec( new TimestampSpec(TIME_COLUMN, "auto", null), new DimensionsSpec( DimensionsSpec.getDefaultSchemas(ImmutableList.of(DIM_NAME)), ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME), - ImmutableList.of() - ), - null, - null + ImmutableList.of() + ) ) ); @@ -533,6 +537,42 @@ public void simpleFirehoseReadingTest() throws IOException Assert.assertEquals((int) MAX_SHARD_NUMBER * MAX_ROWS, (int) rowcount); } + @Test + public void testTransformSpec() throws IOException + { + Assert.assertEquals(MAX_SHARD_NUMBER.longValue(), segmentSet.size()); + Integer rowcount = 0; + final TransformSpec transformSpec = new TransformSpec( + new SelectorDimFilter(Column.TIME_COLUMN_NAME, "1", null), + ImmutableList.of( + new Transform(METRIC_FLOAT_NAME, METRIC_FLOAT_NAME + " * 10", ExprMacroTable.nil()) + ) + ); + int skipped = 0; + try (final IngestSegmentFirehose firehose = + (IngestSegmentFirehose) + factory.connect(transformSpec.decorate(rowParser), null)) { + while (firehose.hasMore()) { + InputRow row = firehose.nextRow(); + if (row == null) { + skipped ++; + continue; + } + Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); + Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray()); + Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getLongMetric(METRIC_LONG_NAME)); + Assert.assertEquals( + METRIC_FLOAT_VALUE * 10, + row.getFloatMetric(METRIC_FLOAT_NAME), + METRIC_FLOAT_VALUE * 0.0001 + ); + ++rowcount; + } + } + Assert.assertEquals(90, skipped); + Assert.assertEquals((int) MAX_ROWS, (int) rowcount); + } + private static ServiceEmitter newMockEmitter() { return new NoopServiceEmitter(); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 0dcfb2afc249..a15306e23980 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -129,6 +129,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.net.URI; @@ -257,6 +258,7 @@ public boolean hasMore() return true; } + @Nullable @Override public InputRow nextRow() { @@ -311,6 +313,7 @@ public boolean hasMore() return inputRowIterator.hasNext(); } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/main/java/io/druid/segment/indexing/DataSchema.java b/server/src/main/java/io/druid/segment/indexing/DataSchema.java index b5d075610a4b..7f4cac4f26ff 100644 --- a/server/src/main/java/io/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/io/druid/segment/indexing/DataSchema.java @@ -187,6 +187,11 @@ public DataSchema withGranularitySpec(GranularitySpec granularitySpec) return new DataSchema(dataSource, parser, aggregators, granularitySpec, transformSpec, jsonMapper); } + public DataSchema withTransformSpec(TransformSpec transformSpec) + { + return new DataSchema(dataSource, parser, aggregators, granularitySpec, transformSpec, jsonMapper); + } + @Override public String toString() { diff --git a/server/src/main/java/io/druid/segment/indexing/TransformSpec.java b/server/src/main/java/io/druid/segment/indexing/TransformSpec.java index aead6fb2202e..dddbcd37a047 100644 --- a/server/src/main/java/io/druid/segment/indexing/TransformSpec.java +++ b/server/src/main/java/io/druid/segment/indexing/TransformSpec.java @@ -58,6 +58,21 @@ public TransformSpec( } } + public static TransformSpec fromInputRowParser(final InputRowParser parser) + { + // Hack: some firehoses and input specs must extract transformSpec from the parser, since they do not + // actually use the parser, but still must respect the transformSpec. This method should extract whatever + // transformSpec "decorate" had put in. + + if (parser instanceof TransformingInputRowParser) { + return ((TransformingInputRowParser) parser).getTransformSpec(); + } else if (parser instanceof TransformingStringInputRowParser) { + return ((TransformingStringInputRowParser) parser).getTransformSpec(); + } else { + return TransformSpec.NONE; + } + } + @JsonProperty @Nullable public DimFilter getFilter() diff --git a/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java b/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java index d38ee9011b62..6fbe05bf1e01 100644 --- a/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java +++ b/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java @@ -53,4 +53,9 @@ public InputRowParser withParseSpec(final ParseSpec parseSpec) { return new TransformingInputRowParser<>(parser.withParseSpec(parseSpec), transformSpec); } + + public TransformSpec getTransformSpec() + { + return transformSpec; + } } diff --git a/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java b/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java index d4a911db0e2f..9a346298fbe8 100644 --- a/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java +++ b/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java @@ -60,4 +60,9 @@ public StringInputRowParser withParseSpec(final ParseSpec parseSpec) { return new TransformingStringInputRowParser(getParseSpec(), getEncoding(), transformSpec); } + + public TransformSpec getTransformSpec() + { + return transformSpec; + } } diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java index 4dc3c9627528..4267f6409a29 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/CombiningFirehoseFactory.java @@ -29,6 +29,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.impl.InputRowParser; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Iterator; @@ -110,6 +111,7 @@ public boolean hasMore() return currentFirehose.hasMore(); } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java index 80cd4165f650..35f70b778ad8 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/EventReceiverFirehoseFactory.java @@ -38,7 +38,7 @@ import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; -import io.druid.data.input.impl.MapInputRowParser; +import io.druid.data.input.impl.InputRowParser; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.DateTimes; @@ -53,6 +53,7 @@ import io.druid.server.security.ResourceType; import org.joda.time.DateTime; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; import javax.ws.rs.POST; @@ -79,7 +80,7 @@ * Builds firehoses that accept events through the {@link EventReceiver} interface. Can also register these * firehoses with an {@link ServiceAnnouncingChatHandlerProvider}. */ -public class EventReceiverFirehoseFactory implements FirehoseFactory +public class EventReceiverFirehoseFactory implements FirehoseFactory>> { private static final EmittingLogger log = new EmittingLogger(EventReceiverFirehoseFactory.class); private static final int DEFAULT_BUFFER_SIZE = 100000; @@ -115,7 +116,10 @@ public EventReceiverFirehoseFactory( } @Override - public Firehose connect(MapInputRowParser firehoseParser, File temporaryDirectory) throws IOException + public Firehose connect( + InputRowParser> firehoseParser, + File temporaryDirectory + ) throws IOException { log.info("Connecting firehose: %s", serviceName); final EventReceiverFirehose firehose = new EventReceiverFirehose(firehoseParser); @@ -151,7 +155,7 @@ public class EventReceiverFirehose implements ChatHandler, Firehose, EventReceiv { private final ScheduledExecutorService exec; private final BlockingQueue buffer; - private final MapInputRowParser parser; + private final InputRowParser> parser; private final Object readLock = new Object(); @@ -160,7 +164,7 @@ public class EventReceiverFirehose implements ChatHandler, Firehose, EventReceiv private final AtomicLong bytesReceived = new AtomicLong(0); private final AtomicLong lastBufferAddFailMsgTime = new AtomicLong(0); - public EventReceiverFirehose(MapInputRowParser parser) + public EventReceiverFirehose(InputRowParser> parser) { this.buffer = new ArrayBlockingQueue<>(bufferSize); this.parser = parser; @@ -180,7 +184,7 @@ public Response addAll( req, new ResourceAction( new Resource("STATE", ResourceType.STATE), - Action.WRITE + Action.WRITE ), authorizerMapper ); @@ -253,6 +257,7 @@ public boolean hasMore() } } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java index fbf498ce5bac..dd21bda0bfdf 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/FixedCountFirehoseFactory.java @@ -27,6 +27,7 @@ import io.druid.data.input.InputRow; import io.druid.data.input.impl.InputRowParser; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; @@ -74,6 +75,7 @@ public boolean hasMore() return i < count && delegateFirehose.hasMore(); } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java b/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java index 2d114ef66b86..90d265f74db5 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IngestSegmentFirehose.java @@ -43,6 +43,8 @@ import io.druid.segment.column.Column; import io.druid.segment.data.IndexedInts; import io.druid.segment.filter.Filters; +import io.druid.segment.indexing.TransformSpec; +import io.druid.segment.indexing.Transformer; import io.druid.utils.Runnables; import javax.annotation.Nullable; @@ -53,15 +55,19 @@ public class IngestSegmentFirehose implements Firehose { + private final Transformer transformer; private Yielder rowYielder; public IngestSegmentFirehose( final List adapters, + final TransformSpec transformSpec, final List dims, final List metrics, final DimFilter dimFilter ) { + this.transformer = transformSpec.toTransformer(); + Sequence rows = Sequences.concat( Iterables.transform( adapters, new Function>() @@ -181,12 +187,13 @@ public boolean hasMore() return !rowYielder.isDone(); } + @Nullable @Override public InputRow nextRow() { final InputRow inputRow = rowYielder.get(); rowYielder = rowYielder.next(null); - return inputRow; + return transformer.transform(inputRow); } @Override diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java index d1fd632672ce..456790533d0f 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/IrcFirehoseFactory.java @@ -33,11 +33,13 @@ import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; +import io.druid.data.input.impl.InputRowParser; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.List; @@ -61,7 +63,7 @@ * ); * } */ -public class IrcFirehoseFactory implements FirehoseFactory +public class IrcFirehoseFactory implements FirehoseFactory>> { private static final Logger log = new Logger(IrcFirehoseFactory.class); @@ -101,7 +103,10 @@ public List getChannels() } @Override - public Firehose connect(final IrcInputRowParser firehoseParser, File temporaryDirectory) throws IOException + public Firehose connect( + final InputRowParser> firehoseParser, + final File temporaryDirectory + ) throws IOException { final IRCApi irc = new IRCApiImpl(false); final LinkedBlockingQueue> queue = new LinkedBlockingQueue>(); @@ -212,6 +217,7 @@ public boolean hasMore() } } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java b/server/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java index 1310d993c357..7b216b4ced4f 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/PredicateFirehose.java @@ -25,6 +25,7 @@ import io.druid.data.input.InputRow; import io.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import java.io.IOException; /** @@ -71,6 +72,7 @@ public boolean hasMore() return false; } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java b/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java index 5ac8a57cbb93..9d20e2557391 100644 --- a/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java +++ b/server/src/main/java/io/druid/segment/realtime/firehose/TimedShutoffFirehoseFactory.java @@ -29,6 +29,7 @@ import io.druid.data.input.impl.InputRowParser; import org.joda.time.DateTime; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.concurrent.ScheduledExecutorService; @@ -102,6 +103,7 @@ public boolean hasMore() return firehose.hasMore(); } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/Plumbers.java b/server/src/main/java/io/druid/segment/realtime/plumber/Plumbers.java index d1053ea2a1da..5826e5f38e3f 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/Plumbers.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/Plumbers.java @@ -20,7 +20,6 @@ package io.druid.segment.realtime.plumber; import com.google.common.base.Supplier; - import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.InputRow; @@ -62,13 +61,9 @@ public static void addNextRow( } if (inputRow == null) { - if (reportParseExceptions) { - throw new ParseException("null input row"); - } else { - log.debug("Discarded null input row, considering unparseable."); - metrics.incrementUnparseable(); - return; - } + log.debug("Discarded null row, considering thrownAway."); + metrics.incrementThrownAway(); + return; } final int numRows; diff --git a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java index c44f7b80efe9..0cfb3338ccb3 100644 --- a/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java +++ b/server/src/test/java/io/druid/realtime/firehose/CombiningFirehoseFactoryTest.java @@ -33,6 +33,7 @@ import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Arrays; @@ -145,6 +146,7 @@ public boolean hasMore() return iterator.hasNext(); } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index cb3489209456..323e82922eb2 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -874,6 +874,7 @@ public boolean hasMore() return rows.hasNext(); } + @Nullable @Override public InputRow nextRow() { diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index 0d983788f39f..67ca7508d7ef 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -47,6 +47,7 @@ import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IncrementalIndexStorageAdapter; +import io.druid.segment.indexing.TransformSpec; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -117,6 +118,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); final IngestSegmentFirehose firehose = new IngestSegmentFirehose( ImmutableList.of(wsa, wsa), + TransformSpec.NONE, ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), null @@ -149,6 +151,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception // Do a spatial filter final IngestSegmentFirehose firehose2 = new IngestSegmentFirehose( ImmutableList.of(new WindowedStorageAdapter(queryable, Intervals.of("2000/3000"))), + TransformSpec.NONE, ImmutableList.of("host", "spatial"), ImmutableList.of("visited_sum", "unique_hosts"), new SpatialDimFilter("spatial", new RadiusBound(new float[]{1, 0}, 0.1f)) From dcd01f184e5b9ceabf550657f8060c89fa2917a1 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 11:51:05 -0700 Subject: [PATCH 06/23] Fix bad merge. --- .../java/io/druid/data/input/MapBasedRow.java | 29 +---------- .../main/java/io/druid/data/input/Rows.java | 50 ++++++++++++++++++- .../IngestSegmentFirehoseFactoryTest.java | 4 +- .../druid/segment/indexing/Transformer.java | 41 ++++++--------- .../segment/indexing/TransformSpecTest.java | 5 +- 5 files changed, 70 insertions(+), 59 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/MapBasedRow.java b/api/src/main/java/io/druid/data/input/MapBasedRow.java index 8e5af6541d00..c1306b42c900 100644 --- a/api/src/main/java/io/druid/data/input/MapBasedRow.java +++ b/api/src/main/java/io/druid/data/input/MapBasedRow.java @@ -22,10 +22,8 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.Lists; -import com.google.common.primitives.Longs; import io.druid.guice.annotations.PublicApi; import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.parsers.ParseException; import org.joda.time.DateTime; @@ -113,37 +111,12 @@ public Number getMetric(String metric) if (metricValue instanceof Number) { return (Number) metricValue; } else if (metricValue instanceof String) { - try { - String metricValueString = StringUtils.removeChar(((String) metricValue).trim(), ','); - // Longs.tryParse() doesn't support leading '+', so we need to trim it ourselves - metricValueString = trimLeadingPlusOfLongString(metricValueString); - Long v = Longs.tryParse(metricValueString); - // Do NOT use ternary operator here, because it makes Java to convert Long to Double - if (v != null) { - return v; - } else { - return Double.valueOf(metricValueString); - } - } - catch (Exception e) { - throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", metric, metricValue); - } + return Rows.stringToNumber(metric, (String) metricValue); } else { throw new ParseException("Unknown type[%s]", metricValue.getClass()); } } - private static String trimLeadingPlusOfLongString(String metricValueString) - { - if (metricValueString.length() > 1 && metricValueString.charAt(0) == '+') { - char secondChar = metricValueString.charAt(1); - if (secondChar >= '0' && secondChar <= '9') { - metricValueString = metricValueString.substring(1); - } - } - return metricValueString; - } - @Override public String toString() { diff --git a/api/src/main/java/io/druid/data/input/Rows.java b/api/src/main/java/io/druid/data/input/Rows.java index a31d1b3a2240..423e0fef6ee0 100644 --- a/api/src/main/java/io/druid/data/input/Rows.java +++ b/api/src/main/java/io/druid/data/input/Rows.java @@ -19,9 +19,13 @@ package io.druid.data.input; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Maps; +import com.google.common.primitives.Longs; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.parsers.ParseException; import java.util.List; import java.util.Map; @@ -33,7 +37,8 @@ public class Rows { /** * @param timeStamp rollup up timestamp to be used to create group key - * @param inputRow input row + * @param inputRow input row + * * @return groupKey for the given input row */ public static List toGroupKey(long timeStamp, InputRow inputRow) @@ -50,4 +55,47 @@ public static List toGroupKey(long timeStamp, InputRow inputRow) dims ); } + + /** + * Convert a string to a number. Throws ParseException + * + * @param name name of the string being parsed (may be used for exception messages) + * @param s the actual string being parsed + * + * @return a number + * + * @throws NullPointerException if the string is null + * @throws ParseException if the string cannot be parsed as a number + */ + public static Number stringToNumber(final String name, final String s) + { + Preconditions.checkNotNull(s, "s"); + + try { + String metricValueString = StringUtils.removeChar(s.trim(), ','); + // Longs.tryParse() doesn't support leading '+', so we need to trim it ourselves + metricValueString = trimLeadingPlusOfLongString(metricValueString); + Long v = Longs.tryParse(metricValueString); + // Do NOT use ternary operator here, because it makes Java to convert Long to Double + if (v != null) { + return v; + } else { + return Double.valueOf(metricValueString); + } + } + catch (Exception e) { + throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", name, s); + } + } + + private static String trimLeadingPlusOfLongString(String metricValueString) + { + if (metricValueString.length() > 1 && metricValueString.charAt(0) == '+') { + char secondChar = metricValueString.charAt(1); + if (secondChar >= '0' && secondChar <= '9') { + metricValueString = metricValueString.substring(1); + } + } + return metricValueString; + } } diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index bf8b64a79e2f..0f51da1f3c2f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -560,10 +560,10 @@ public void testTransformSpec() throws IOException } Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray()); - Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getLongMetric(METRIC_LONG_NAME)); + Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME).longValue()); Assert.assertEquals( METRIC_FLOAT_VALUE * 10, - row.getFloatMetric(METRIC_FLOAT_NAME), + row.getMetric(METRIC_FLOAT_NAME).floatValue(), METRIC_FLOAT_VALUE * 0.0001 ); ++rowcount; diff --git a/server/src/main/java/io/druid/segment/indexing/Transformer.java b/server/src/main/java/io/druid/segment/indexing/Transformer.java index 1d90704256b8..f1b5d13b8899 100644 --- a/server/src/main/java/io/druid/segment/indexing/Transformer.java +++ b/server/src/main/java/io/druid/segment/indexing/Transformer.java @@ -22,8 +22,11 @@ import com.google.common.base.Strings; import io.druid.data.input.InputRow; import io.druid.data.input.Row; +import io.druid.data.input.Rows; import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.ISE; import io.druid.math.expr.Expr; +import io.druid.math.expr.ExprEval; import io.druid.query.filter.ValueMatcher; import io.druid.query.groupby.RowBasedColumnSelectorFactory; import io.druid.segment.column.Column; @@ -162,35 +165,23 @@ public Object getRaw(final String column) } @Override - public float getFloatMetric(final String metric) + public Number getMetric(final String metric) { final Expr transform = transforms.get(metric); if (transform != null) { - return (float) transform.eval(this::getValueFromRow).asDouble(); - } else { - return row.getFloatMetric(metric); - } - } - - @Override - public long getLongMetric(final String metric) - { - final Expr transform = transforms.get(metric); - if (transform != null) { - return transform.eval(this::getValueFromRow).asLong(); - } else { - return row.getLongMetric(metric); - } - } - - @Override - public double getDoubleMetric(final String metric) - { - final Expr transform = transforms.get(metric); - if (transform != null) { - return transform.eval(this::getValueFromRow).asDouble(); + final ExprEval eval = transform.eval(this::getValueFromRow); + switch (eval.type()) { + case DOUBLE: + return eval.asDouble(); + case LONG: + return eval.asLong(); + case STRING: + return Rows.stringToNumber(metric, eval.asString()); + default: + throw new ISE("WTF, unexpected eval type[%s]", eval.type()); + } } else { - return row.getDoubleMetric(metric); + return row.getMetric(metric); } } diff --git a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java index 3e299d6812ea..9e58243772a4 100644 --- a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java +++ b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java @@ -83,13 +83,12 @@ public void testTransforms() Assert.assertEquals(DateTimes.of("2000-01-01"), row.getTimestamp()); Assert.assertEquals(ImmutableList.of("f", "x", "y"), row.getDimensions()); Assert.assertEquals(ImmutableList.of("foo"), row.getDimension("x")); - Assert.assertEquals(3.0, row.getDoubleMetric("b"), 0); + Assert.assertEquals(3.0, row.getMetric("b").doubleValue(), 0); Assert.assertEquals("foobar", row.getRaw("f")); Assert.assertEquals(ImmutableList.of("foobar"), row.getDimension("f")); Assert.assertEquals(ImmutableList.of("5.0"), row.getDimension("g")); Assert.assertEquals(ImmutableList.of(), row.getDimension("h")); - Assert.assertEquals(0L, row.getLongMetric("f")); - Assert.assertEquals(5L, row.getLongMetric("g")); + Assert.assertEquals(5L, row.getMetric("g").longValue()); } @Test From 577326f59e5d0bcd234fba38bad5b2a19f56700d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 11:59:42 -0700 Subject: [PATCH 07/23] Adjust imports. --- .../java/io/druid/indexer/HadoopDruidIndexerMapperTest.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java index 34ff0421d618..742aa4ff903a 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Maps; import io.druid.data.input.InputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; @@ -50,7 +49,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; From 1511c6dd9fbf960a3e808ad4a10c374635388d83 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 12:11:32 -0700 Subject: [PATCH 08/23] Adjust whitespace. --- .../indexing/firehose/IngestSegmentFirehoseFactoryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 0f51da1f3c2f..8656422d25b6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -555,7 +555,7 @@ public void testTransformSpec() throws IOException while (firehose.hasMore()) { InputRow row = firehose.nextRow(); if (row == null) { - skipped ++; + skipped++; continue; } Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray()); From 7c06024bb534136d158b92dce1bf54ad64905fb6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 13:04:10 -0700 Subject: [PATCH 09/23] Make Transform into an interface. --- .../java/io/druid/data/input/MapBasedRow.java | 30 +---- .../main/java/io/druid/data/input/Rows.java | 73 ++++++++--- .../indexing/kafka/KafkaIndexTaskTest.java | 15 +-- .../indexer/HadoopDruidIndexerMapperTest.java | 4 +- .../indexing/common/task/IndexTaskTest.java | 4 +- .../common/task/RealtimeIndexTaskTest.java | 16 +-- .../IngestSegmentFirehoseFactoryTest.java | 4 +- .../segment/indexing/ExpressionTransform.java | 124 ++++++++++++++++++ .../druid/segment/indexing/RowFunction.java | 30 +++++ .../io/druid/segment/indexing/Transform.java | 81 ++---------- .../druid/segment/indexing/Transformer.java | 49 ++----- .../segment/indexing/DataSchemaTest.java | 2 +- .../segment/indexing/TransformSpecTest.java | 40 +++++- .../appenderator/AppenderatorTester.java | 13 +- 14 files changed, 285 insertions(+), 200 deletions(-) create mode 100644 server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java create mode 100644 server/src/main/java/io/druid/segment/indexing/RowFunction.java diff --git a/api/src/main/java/io/druid/data/input/MapBasedRow.java b/api/src/main/java/io/druid/data/input/MapBasedRow.java index c1306b42c900..5a7c4db03737 100644 --- a/api/src/main/java/io/druid/data/input/MapBasedRow.java +++ b/api/src/main/java/io/druid/data/input/MapBasedRow.java @@ -21,13 +21,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.collect.Lists; import io.druid.guice.annotations.PublicApi; import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.parsers.ParseException; import org.joda.time.DateTime; -import java.util.Collections; import java.util.List; import java.util.Map; @@ -36,8 +33,6 @@ @PublicApi public class MapBasedRow implements Row { - private static final Long LONG_ZERO = 0L; - private final DateTime timestamp; private final Map event; @@ -81,16 +76,7 @@ public Map getEvent() @Override public List getDimension(String dimension) { - final Object dimValue = event.get(dimension); - - if (dimValue == null) { - return Collections.emptyList(); - } else if (dimValue instanceof List) { - // guava's toString function fails on null objects, so please do not use it - return Lists.transform((List) dimValue, String::valueOf); - } else { - return Collections.singletonList(String.valueOf(dimValue)); - } + return Rows.objectToStrings(event.get(dimension)); } @Override @@ -102,19 +88,7 @@ public Object getRaw(String dimension) @Override public Number getMetric(String metric) { - Object metricValue = event.get(metric); - - if (metricValue == null) { - return LONG_ZERO; - } - - if (metricValue instanceof Number) { - return (Number) metricValue; - } else if (metricValue instanceof String) { - return Rows.stringToNumber(metric, (String) metricValue); - } else { - throw new ParseException("Unknown type[%s]", metricValue.getClass()); - } + return Rows.objectToNumber(metric, event.get(metric)); } @Override diff --git a/api/src/main/java/io/druid/data/input/Rows.java b/api/src/main/java/io/druid/data/input/Rows.java index 423e0fef6ee0..0ef09e9fa246 100644 --- a/api/src/main/java/io/druid/data/input/Rows.java +++ b/api/src/main/java/io/druid/data/input/Rows.java @@ -19,7 +19,6 @@ package io.druid.data.input; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Maps; @@ -27,6 +26,8 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.parsers.ParseException; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -35,6 +36,8 @@ */ public class Rows { + public static final Long LONG_ZERO = 0L; + /** * @param timeStamp rollup up timestamp to be used to create group key * @param inputRow input row @@ -57,34 +60,64 @@ public static List toGroupKey(long timeStamp, InputRow inputRow) } /** - * Convert a string to a number. Throws ParseException + * Convert an object to a list of strings. + */ + public static List objectToStrings(final Object inputValue) + { + if (inputValue == null) { + return Collections.emptyList(); + } else if (inputValue instanceof List) { + // guava's toString function fails on null objects, so please do not use it + final List values = (List) inputValue; + + final List retVal = new ArrayList<>(values.size()); + for (Object val : values) { + retVal.add(String.valueOf(val)); + } + + return retVal; + } else { + return Collections.singletonList(String.valueOf(inputValue)); + } + } + + /** + * Convert an object to a number. Nulls are treated as zeroes. * - * @param name name of the string being parsed (may be used for exception messages) - * @param s the actual string being parsed + * @param name field name of the object being converted (may be used for exception messages) + * @param inputValue the actual object being converted * * @return a number * * @throws NullPointerException if the string is null - * @throws ParseException if the string cannot be parsed as a number + * @throws ParseException if the column cannot be converted to a number */ - public static Number stringToNumber(final String name, final String s) + public static Number objectToNumber(final String name, final Object inputValue) { - Preconditions.checkNotNull(s, "s"); + if (inputValue == null) { + return Rows.LONG_ZERO; + } - try { - String metricValueString = StringUtils.removeChar(s.trim(), ','); - // Longs.tryParse() doesn't support leading '+', so we need to trim it ourselves - metricValueString = trimLeadingPlusOfLongString(metricValueString); - Long v = Longs.tryParse(metricValueString); - // Do NOT use ternary operator here, because it makes Java to convert Long to Double - if (v != null) { - return v; - } else { - return Double.valueOf(metricValueString); + if (inputValue instanceof Number) { + return (Number) inputValue; + } else if (inputValue instanceof String) { + try { + String metricValueString = StringUtils.removeChar(((String) inputValue).trim(), ','); + // Longs.tryParse() doesn't support leading '+', so we need to trim it ourselves + metricValueString = trimLeadingPlusOfLongString(metricValueString); + Long v = Longs.tryParse(metricValueString); + // Do NOT use ternary operator here, because it makes Java to convert Long to Double + if (v != null) { + return v; + } else { + return Double.valueOf(metricValueString); + } } - } - catch (Exception e) { - throw new ParseException(e, "Unable to parse metrics[%s], value[%s]", name, s); + catch (Exception e) { + throw new ParseException(e, "Unable to parse value[%s] for field[%s]", inputValue, name); + } + } else { + throw new ParseException("Unknown type[%s] for field", inputValue.getClass(), inputValue); } } diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 58ad873aad9f..cdfed98985a0 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -37,11 +37,9 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -//CHECKSTYLE.OFF: Regexp import com.metamx.common.logger.Logger; -//CHECKSTYLE.ON: Regexp import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.core.NoopEmitter; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.CacheConfig; @@ -73,7 +71,6 @@ import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.indexing.test.TestDataSegmentAnnouncer; import io.druid.indexing.test.TestDataSegmentKiller; -import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.CompressionUtils; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; @@ -115,7 +112,7 @@ import io.druid.segment.TestHelper; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.ExpressionTransform; import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.DataSegmentPusher; @@ -240,11 +237,7 @@ public static void setupClass() throws Exception emitter = new ServiceEmitter( "service", "host", - new LoggingEmitter( - log, - LoggingEmitter.Level.ERROR, - new DefaultObjectMapper() - ) + new NoopEmitter() ); emitter.start(); EmittingLogger.registerEmitter(emitter); @@ -533,7 +526,7 @@ public void testRunWithTransformSpec() throws Exception new TransformSpec( new SelectorDimFilter("dim1", "b", null), ImmutableList.of( - new Transform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) ) ) ), diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java index 742aa4ff903a..cf33fb83e19b 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -36,7 +36,7 @@ import io.druid.query.filter.SelectorDimFilter; import io.druid.segment.TestHelper; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.ExpressionTransform; import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.hadoop.conf.Configuration; @@ -128,7 +128,7 @@ public void testHadoopyStringParserWithTransformSpec() throws Exception new TransformSpec( new SelectorDimFilter("dim1", "foo", null), ImmutableList.of( - new Transform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) ) ) ), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 6279c8edff4b..e0606a8f8518 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -57,7 +57,7 @@ import io.druid.segment.IndexMergerV9; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; -import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.ExpressionTransform; import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import io.druid.segment.indexing.granularity.GranularitySpec; @@ -236,7 +236,7 @@ public void testTransformSpec() throws Exception new TransformSpec( new SelectorDimFilter("dim", "b", null), ImmutableList.of( - new Transform("dimt", "concat(dim,dim)", ExprMacroTable.nil()) + new ExpressionTransform("dimt", "concat(dim,dim)", ExprMacroTable.nil()) ) ), null, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 65a34bc68106..95f0e94b78e3 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -30,16 +30,13 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -//CHECKSTYLE.OFF: Regexp import com.metamx.common.logger.Logger; -//CHECKSTYLE.ON: Regexp import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.core.NoopEmitter; import com.metamx.emitter.service.ServiceEmitter; import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; -import io.druid.java.util.common.concurrent.Execs; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; import io.druid.data.input.InputRow; @@ -75,6 +72,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.jackson.JacksonUtils; @@ -105,9 +103,9 @@ import io.druid.query.timeseries.TimeseriesResultValue; import io.druid.segment.TestHelper; import io.druid.segment.indexing.DataSchema; +import io.druid.segment.indexing.ExpressionTransform; import io.druid.segment.indexing.RealtimeIOConfig; import io.druid.segment.indexing.RealtimeTuningConfig; -import io.druid.segment.indexing.Transform; import io.druid.segment.indexing.TransformSpec; import io.druid.segment.indexing.granularity.UniformGranularitySpec; import io.druid.segment.loading.SegmentLoaderConfig; @@ -155,11 +153,7 @@ public class RealtimeIndexTaskTest private static final ServiceEmitter emitter = new ServiceEmitter( "service", "host", - new LoggingEmitter( - log, - LoggingEmitter.Level.ERROR, - jsonMapper - ) + new NoopEmitter() ); private static final String FAIL_DIM = "__fail__"; @@ -391,7 +385,7 @@ public void testTransformSpec() throws Exception final TransformSpec transformSpec = new TransformSpec( new SelectorDimFilter("dim1", "foo", null), ImmutableList.of( - new Transform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) + new ExpressionTransform("dim1t", "concat(dim1,dim1)", ExprMacroTable.nil()) ) ); final RealtimeIndexTask task = makeRealtimeTask(null, transformSpec, true, 0); diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 8656422d25b6..94509c085318 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -71,7 +71,7 @@ import io.druid.segment.column.Column; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; -import io.druid.segment.indexing.Transform; +import io.druid.segment.indexing.ExpressionTransform; import io.druid.segment.indexing.TransformSpec; import io.druid.segment.loading.DataSegmentArchiver; import io.druid.segment.loading.DataSegmentKiller; @@ -545,7 +545,7 @@ public void testTransformSpec() throws IOException final TransformSpec transformSpec = new TransformSpec( new SelectorDimFilter(Column.TIME_COLUMN_NAME, "1", null), ImmutableList.of( - new Transform(METRIC_FLOAT_NAME, METRIC_FLOAT_NAME + " * 10", ExprMacroTable.nil()) + new ExpressionTransform(METRIC_FLOAT_NAME, METRIC_FLOAT_NAME + " * 10", ExprMacroTable.nil()) ) ); int skipped = 0; diff --git a/server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java b/server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java new file mode 100644 index 000000000000..40d8cd844196 --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java @@ -0,0 +1,124 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import io.druid.data.input.Row; +import io.druid.math.expr.Expr; +import io.druid.math.expr.ExprMacroTable; +import io.druid.math.expr.Parser; +import io.druid.segment.column.Column; + +import java.util.Objects; + +public class ExpressionTransform implements Transform +{ + private final String name; + private final String expression; + private final ExprMacroTable macroTable; + + @JsonCreator + public ExpressionTransform( + @JsonProperty("name") final String name, + @JsonProperty("expression") final String expression, + @JacksonInject ExprMacroTable macroTable + ) + { + this.name = Preconditions.checkNotNull(name, "name"); + this.expression = Preconditions.checkNotNull(expression, "expression"); + this.macroTable = macroTable; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getExpression() + { + return expression; + } + + @Override + public RowFunction getRowFunction() + { + final Expr expr = Parser.parse(expression, Preconditions.checkNotNull(this.macroTable, "macroTable")); + return new ExpressionRowFunction(expr); + } + + static class ExpressionRowFunction implements RowFunction + { + private final Expr expr; + + ExpressionRowFunction(final Expr expr) + { + this.expr = expr; + } + + @Override + public Object eval(final Row row) + { + return expr.eval(name -> getValueFromRow(row, name)).value(); + } + } + + private static Object getValueFromRow(final Row row, final String column) + { + if (column.equals(Column.TIME_COLUMN_NAME)) { + return row.getTimestampFromEpoch(); + } else { + return row.getRaw(column); + } + } + + @Override + public boolean equals(final Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ExpressionTransform that = (ExpressionTransform) o; + return Objects.equals(name, that.name) && + Objects.equals(expression, that.expression); + } + + @Override + public int hashCode() + { + return Objects.hash(name, expression); + } + + @Override + public String toString() + { + return "ExpressionTransform{" + + "name='" + name + '\'' + + ", expression='" + expression + '\'' + + '}'; + } +} diff --git a/server/src/main/java/io/druid/segment/indexing/RowFunction.java b/server/src/main/java/io/druid/segment/indexing/RowFunction.java new file mode 100644 index 000000000000..c9dabd744c57 --- /dev/null +++ b/server/src/main/java/io/druid/segment/indexing/RowFunction.java @@ -0,0 +1,30 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.segment.indexing; + +import io.druid.data.input.Row; + +/** + * Interface for evaluating functions on rows. Used by {@link Transformer}. + */ +public interface RowFunction +{ + Object eval(Row row); +} diff --git a/server/src/main/java/io/druid/segment/indexing/Transform.java b/server/src/main/java/io/druid/segment/indexing/Transform.java index ed64b8348985..cb8db5b3c9f6 100644 --- a/server/src/main/java/io/druid/segment/indexing/Transform.java +++ b/server/src/main/java/io/druid/segment/indexing/Transform.java @@ -19,77 +19,16 @@ package io.druid.segment.indexing; -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import io.druid.math.expr.Expr; -import io.druid.math.expr.ExprMacroTable; -import io.druid.math.expr.Parser; - -import java.util.Objects; - -public class Transform +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "expression", value = ExpressionTransform.class) +}) +public interface Transform { - private final String name; - private final String expression; - private final ExprMacroTable macroTable; - - @JsonCreator - public Transform( - @JsonProperty("name") final String name, - @JsonProperty("expression") final String expression, - @JacksonInject ExprMacroTable macroTable - ) - { - this.name = Preconditions.checkNotNull(name, "name"); - this.expression = Preconditions.checkNotNull(expression, "expression"); - this.macroTable = macroTable; - } - - @JsonProperty - public String getName() - { - return name; - } - - @JsonProperty - public String getExpression() - { - return expression; - } - - public Expr toExpr() - { - return Parser.parse(expression, Preconditions.checkNotNull(macroTable, "macroTable")); - } - - @Override - public boolean equals(final Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final Transform transform = (Transform) o; - return Objects.equals(name, transform.name) && - Objects.equals(expression, transform.expression); - } - - @Override - public int hashCode() - { - return Objects.hash(name, expression); - } + String getName(); - @Override - public String toString() - { - return "Transform{" + - "name='" + name + '\'' + - ", expression='" + expression + '\'' + - '}'; - } + RowFunction getRowFunction(); } diff --git a/server/src/main/java/io/druid/segment/indexing/Transformer.java b/server/src/main/java/io/druid/segment/indexing/Transformer.java index f1b5d13b8899..78f273e914e9 100644 --- a/server/src/main/java/io/druid/segment/indexing/Transformer.java +++ b/server/src/main/java/io/druid/segment/indexing/Transformer.java @@ -19,21 +19,16 @@ package io.druid.segment.indexing; -import com.google.common.base.Strings; import io.druid.data.input.InputRow; import io.druid.data.input.Row; import io.druid.data.input.Rows; import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.ISE; -import io.druid.math.expr.Expr; -import io.druid.math.expr.ExprEval; import io.druid.query.filter.ValueMatcher; import io.druid.query.groupby.RowBasedColumnSelectorFactory; import io.druid.segment.column.Column; import org.joda.time.DateTime; import javax.annotation.Nullable; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -44,14 +39,14 @@ */ public class Transformer { - private final Map transforms = new HashMap<>(); + private final Map transforms = new HashMap<>(); private final ThreadLocal rowSupplierForValueMatcher = new ThreadLocal<>(); private final ValueMatcher valueMatcher; Transformer(final TransformSpec transformSpec) { for (final Transform transform : transformSpec.getTransforms()) { - transforms.put(transform.getName(), transform.toExpr()); + transforms.put(transform.getName(), transform.getRowFunction()); } if (transformSpec.getFilter() != null) { @@ -100,9 +95,9 @@ public InputRow transform(@Nullable final InputRow row) public static class TransformedInputRow implements InputRow { private final InputRow row; - private final Map transforms; + private final Map transforms; - public TransformedInputRow(final InputRow row, final Map transforms) + public TransformedInputRow(final InputRow row, final Map transforms) { this.row = row; this.transforms = transforms; @@ -117,9 +112,9 @@ public List getDimensions() @Override public long getTimestampFromEpoch() { - final Expr transform = transforms.get(Column.TIME_COLUMN_NAME); + final RowFunction transform = transforms.get(Column.TIME_COLUMN_NAME); if (transform != null) { - return transform.eval(this::getValueFromRow).asLong(); + return Rows.objectToNumber(Column.TIME_COLUMN_NAME, transform.eval(row)).longValue(); } else { return row.getTimestampFromEpoch(); } @@ -128,9 +123,9 @@ public long getTimestampFromEpoch() @Override public DateTime getTimestamp() { - final Expr transform = transforms.get(Column.TIME_COLUMN_NAME); + final RowFunction transform = transforms.get(Column.TIME_COLUMN_NAME); if (transform != null) { - return DateTimes.utc(transform.eval(this::getValueFromRow).asLong()); + return DateTimes.utc(getTimestampFromEpoch()); } else { return row.getTimestamp(); } @@ -139,15 +134,9 @@ public DateTime getTimestamp() @Override public List getDimension(final String dimension) { - final Expr transform = transforms.get(dimension); + final RowFunction transform = transforms.get(dimension); if (transform != null) { - // Always return single-value. Expressions don't support array/list operations yet. - final String s = transform.eval(this::getValueFromRow).asString(); - if (Strings.isNullOrEmpty(s)) { - return Collections.emptyList(); - } else { - return Collections.singletonList(s); - } + return Rows.objectToStrings(transform.eval(row)); } else { return row.getDimension(dimension); } @@ -156,9 +145,9 @@ public List getDimension(final String dimension) @Override public Object getRaw(final String column) { - final Expr transform = transforms.get(column); + final RowFunction transform = transforms.get(column); if (transform != null) { - return transform.eval(this::getValueFromRow).value(); + return transform.eval(row); } else { return row.getRaw(column); } @@ -167,19 +156,9 @@ public Object getRaw(final String column) @Override public Number getMetric(final String metric) { - final Expr transform = transforms.get(metric); + final RowFunction transform = transforms.get(metric); if (transform != null) { - final ExprEval eval = transform.eval(this::getValueFromRow); - switch (eval.type()) { - case DOUBLE: - return eval.asDouble(); - case LONG: - return eval.asLong(); - case STRING: - return Rows.stringToNumber(metric, eval.asString()); - default: - throw new ISE("WTF, unexpected eval type[%s]", eval.type()); - } + return Rows.objectToNumber(metric, transform.eval(row)); } else { return row.getMetric(metric); } diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java index 1b77f141ea35..abca29154ae8 100644 --- a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -158,7 +158,7 @@ public void testTransformSpec() throws Exception new TransformSpec( new SelectorDimFilter("dimA", "foo", null), ImmutableList.of( - new Transform("expr", "concat(dimA,dimA)", TestExprMacroTable.INSTANCE) + new ExpressionTransform("expr", "concat(dimA,dimA)", TestExprMacroTable.INSTANCE) ) ), jsonMapper diff --git a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java index 9e58243772a4..b9a7cede7113 100644 --- a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java +++ b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java @@ -19,6 +19,7 @@ package io.druid.segment.indexing; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.druid.data.input.InputRow; @@ -31,6 +32,7 @@ import io.druid.query.expression.TestExprMacroTable; import io.druid.query.filter.AndDimFilter; import io.druid.query.filter.SelectorDimFilter; +import io.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; @@ -69,9 +71,9 @@ public void testTransforms() final TransformSpec transformSpec = new TransformSpec( null, ImmutableList.of( - new Transform("f", "concat(x,y)", TestExprMacroTable.INSTANCE), - new Transform("g", "a + b", TestExprMacroTable.INSTANCE), - new Transform("h", "concat(f,g)", TestExprMacroTable.INSTANCE) + new ExpressionTransform("f", "concat(x,y)", TestExprMacroTable.INSTANCE), + new ExpressionTransform("g", "a + b", TestExprMacroTable.INSTANCE), + new ExpressionTransform("h", "concat(f,g)", TestExprMacroTable.INSTANCE) ) ); @@ -103,8 +105,8 @@ public void testFilterOnTransforms() ) ), ImmutableList.of( - new Transform("f", "concat(x,y)", TestExprMacroTable.INSTANCE), - new Transform("g", "a + b", TestExprMacroTable.INSTANCE) + new ExpressionTransform("f", "concat(x,y)", TestExprMacroTable.INSTANCE), + new ExpressionTransform("g", "a + b", TestExprMacroTable.INSTANCE) ) ); @@ -119,7 +121,7 @@ public void testTransformTimeFromOtherFields() final TransformSpec transformSpec = new TransformSpec( null, ImmutableList.of( - new Transform("__time", "(a + b) * 3600000", TestExprMacroTable.INSTANCE) + new ExpressionTransform("__time", "(a + b) * 3600000", TestExprMacroTable.INSTANCE) ) ); @@ -137,7 +139,7 @@ public void testTransformTimeFromTime() final TransformSpec transformSpec = new TransformSpec( null, ImmutableList.of( - new Transform("__time", "__time + 3600000", TestExprMacroTable.INSTANCE) + new ExpressionTransform("__time", "__time + 3600000", TestExprMacroTable.INSTANCE) ) ); @@ -148,4 +150,28 @@ public void testTransformTimeFromTime() Assert.assertEquals(DateTimes.of("2000-01-01T01:00:00Z"), row.getTimestamp()); Assert.assertEquals(DateTimes.of("2000-01-01T01:00:00Z").getMillis(), row.getTimestampFromEpoch()); } + + @Test + public void testSerde() throws Exception + { + final TransformSpec transformSpec = new TransformSpec( + new AndDimFilter( + ImmutableList.of( + new SelectorDimFilter("x", "foo", null), + new SelectorDimFilter("f", "foobar", null), + new SelectorDimFilter("g", "5.0", null) + ) + ), + ImmutableList.of( + new ExpressionTransform("f", "concat(x,y)", TestExprMacroTable.INSTANCE), + new ExpressionTransform("g", "a + b", TestExprMacroTable.INSTANCE) + ) + ); + + final ObjectMapper jsonMapper = TestHelper.getJsonMapper(); + Assert.assertEquals( + transformSpec, + jsonMapper.readValue(jsonMapper.writeValueAsString(transformSpec), TransformSpec.class) + ); + } } diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index a6b75f3e035b..766fa19709b7 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -21,20 +21,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -//CHECKSTYLE.OFF: Regexp -import com.metamx.common.logger.Logger; -//CHECKSTYLE.ON: Regexp import com.metamx.emitter.EmittingLogger; -import com.metamx.emitter.core.LoggingEmitter; +import com.metamx.emitter.core.NoopEmitter; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.MapCache; -import io.druid.java.util.common.concurrent.Execs; import io.druid.data.input.impl.DimensionsSpec; import io.druid.data.input.impl.JSONParseSpec; import io.druid.data.input.impl.MapInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.IntervalChunkingQueryRunnerDecorator; @@ -171,11 +168,7 @@ public int columnCacheSizeBytes() emitter = new ServiceEmitter( "test", "test", - new LoggingEmitter( - new Logger(AppenderatorTester.class), - LoggingEmitter.Level.INFO, - objectMapper - ) + new NoopEmitter() ); emitter.start(); EmittingLogger.registerEmitter(emitter); From 8a61b6f49bcb9d066730e820c9f7f92d41719d54 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 14:07:45 -0700 Subject: [PATCH 10/23] Add missing annotation. --- .../main/java/io/druid/segment/indexing/ExpressionTransform.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java b/server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java index 40d8cd844196..7369a2fce8c5 100644 --- a/server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java +++ b/server/src/main/java/io/druid/segment/indexing/ExpressionTransform.java @@ -50,6 +50,7 @@ public ExpressionTransform( } @JsonProperty + @Override public String getName() { return name; From 4f724cc5a09996f5ab9ad0b51974f6d4d5052242 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 15:23:16 -0700 Subject: [PATCH 11/23] Switch logger. --- .../io/druid/indexing/common/task/RealtimeIndexTaskTest.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 95f0e94b78e3..6d15801f7acb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -30,7 +30,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.metamx.common.logger.Logger; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.NoopEmitter; import com.metamx.emitter.service.ServiceEmitter; @@ -76,6 +75,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.jackson.JacksonUtils; +import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; import io.druid.math.expr.ExprMacroTable; import io.druid.metadata.EntryExistsException; @@ -149,7 +149,6 @@ public class RealtimeIndexTaskTest { private static final Logger log = new Logger(RealtimeIndexTaskTest.class); - private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); private static final ServiceEmitter emitter = new ServiceEmitter( "service", "host", From 53e0efaba1d4d9764b6f26328809d64fca1b5670 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 15:33:43 -0700 Subject: [PATCH 12/23] Switch logger. --- .../test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index cdfed98985a0..af0574da8a90 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -37,7 +37,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import com.metamx.common.logger.Logger; import com.metamx.emitter.EmittingLogger; import com.metamx.emitter.core.NoopEmitter; import com.metamx.emitter.service.ServiceEmitter; @@ -79,6 +78,7 @@ import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.JSONPathFieldSpec; import io.druid.java.util.common.parsers.JSONPathSpec; import io.druid.math.expr.ExprMacroTable; From 707b49ec2753b45f1740d694e457f19f075d7a20 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 16:47:07 -0700 Subject: [PATCH 13/23] Adjust test. --- .../java/io/druid/segment/realtime/RealtimeManagerTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index be2243062b7c..e82d7146d4a4 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -334,8 +334,8 @@ public void testRun() throws Exception } Assert.assertEquals(1, realtimeManager.getMetrics("test").processed()); - Assert.assertEquals(1, realtimeManager.getMetrics("test").thrownAway()); - Assert.assertEquals(2, realtimeManager.getMetrics("test").unparseable()); + Assert.assertEquals(2, realtimeManager.getMetrics("test").thrownAway()); + Assert.assertEquals(1, realtimeManager.getMetrics("test").unparseable()); Assert.assertTrue(plumber.isStartedJob()); Assert.assertTrue(plumber.isFinishedJob()); Assert.assertEquals(0, plumber.getPersistCount()); From b780be92337c4bfb7ebf465a1f3dc26610f49e9c Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 16 Oct 2017 17:54:33 -0700 Subject: [PATCH 14/23] Adjustment to handling for DatasourceIngestionSpec. --- .../hadoop/DatasourceIngestionSpec.java | 98 ++++++++++++------- .../indexer/hadoop/DatasourceInputFormat.java | 3 +- .../hadoop/DatasourceRecordReader.java | 4 +- .../indexer/path/DatasourcePathSpec.java | 6 +- ...cUpdateDatasourcePathSpecSegmentsTest.java | 14 ++- .../hadoop/DatasourceIngestionSpecTest.java | 11 ++- .../hadoop/DatasourceRecordReaderTest.java | 3 +- .../indexer/path/DatasourcePathSpecTest.java | 3 +- 8 files changed, 87 insertions(+), 55 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java index 87e671f58783..b55bdf4aa7c9 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceIngestionSpec.java @@ -26,10 +26,12 @@ import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.granularity.Granularity; import io.druid.query.filter.DimFilter; +import io.druid.segment.indexing.TransformSpec; import io.druid.timeline.DataSegment; import org.joda.time.Interval; import java.util.List; +import java.util.Objects; public class DatasourceIngestionSpec { @@ -41,6 +43,10 @@ public class DatasourceIngestionSpec private final List metrics; private final boolean ignoreWhenNoSegments; + // Note that the only purpose of the transformSpec field is to hold the value from the overall dataSchema. + // It is not meant to be provided by end users, and will be overwritten. + private final TransformSpec transformSpec; + @JsonCreator public DatasourceIngestionSpec( @JsonProperty("dataSource") String dataSource, @@ -50,7 +56,8 @@ public DatasourceIngestionSpec( @JsonProperty("filter") DimFilter filter, @JsonProperty("dimensions") List dimensions, @JsonProperty("metrics") List metrics, - @JsonProperty("ignoreWhenNoSegments") boolean ignoreWhenNoSegments + @JsonProperty("ignoreWhenNoSegments") boolean ignoreWhenNoSegments, + @JsonProperty("transformSpec") TransformSpec transformSpec ) { this.dataSource = Preconditions.checkNotNull(dataSource, "null dataSource"); @@ -59,7 +66,7 @@ public DatasourceIngestionSpec( interval == null || intervals == null, "please specify intervals only" ); - + List theIntervals = null; if (interval != null) { theIntervals = ImmutableList.of(interval); @@ -78,6 +85,7 @@ public DatasourceIngestionSpec( this.metrics = metrics; this.ignoreWhenNoSegments = ignoreWhenNoSegments; + this.transformSpec = transformSpec != null ? transformSpec : TransformSpec.NONE; } @JsonProperty @@ -122,6 +130,12 @@ public boolean isIgnoreWhenNoSegments() return ignoreWhenNoSegments; } + @JsonProperty + public TransformSpec getTransformSpec() + { + return transformSpec; + } + public DatasourceIngestionSpec withDimensions(List dimensions) { return new DatasourceIngestionSpec( @@ -132,7 +146,8 @@ public DatasourceIngestionSpec withDimensions(List dimensions) filter, dimensions, metrics, - ignoreWhenNoSegments + ignoreWhenNoSegments, + transformSpec ); } @@ -146,7 +161,8 @@ public DatasourceIngestionSpec withMetrics(List metrics) filter, dimensions, metrics, - ignoreWhenNoSegments + ignoreWhenNoSegments, + transformSpec ); } @@ -160,7 +176,8 @@ public DatasourceIngestionSpec withQueryGranularity(Granularity granularity) filter, dimensions, metrics, - ignoreWhenNoSegments + ignoreWhenNoSegments, + transformSpec ); } @@ -174,12 +191,28 @@ public DatasourceIngestionSpec withIgnoreWhenNoSegments(boolean ignoreWhenNoSegm filter, dimensions, metrics, - ignoreWhenNoSegments + ignoreWhenNoSegments, + transformSpec + ); + } + + public DatasourceIngestionSpec withTransformSpec(TransformSpec transformSpec) + { + return new DatasourceIngestionSpec( + dataSource, + null, + intervals, + segments, + filter, + dimensions, + metrics, + ignoreWhenNoSegments, + transformSpec ); } @Override - public boolean equals(Object o) + public boolean equals(final Object o) { if (this == o) { return true; @@ -187,42 +220,30 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - - DatasourceIngestionSpec that = (DatasourceIngestionSpec) o; - - if (ignoreWhenNoSegments != that.ignoreWhenNoSegments) { - return false; - } - if (!dataSource.equals(that.dataSource)) { - return false; - } - if (!intervals.equals(that.intervals)) { - return false; - } - if (segments != null ? !segments.equals(that.segments) : that.segments != null) { - return false; - } - if (filter != null ? !filter.equals(that.filter) : that.filter != null) { - return false; - } - if (dimensions != null ? !dimensions.equals(that.dimensions) : that.dimensions != null) { - return false; - } - return !(metrics != null ? !metrics.equals(that.metrics) : that.metrics != null); - + final DatasourceIngestionSpec that = (DatasourceIngestionSpec) o; + return ignoreWhenNoSegments == that.ignoreWhenNoSegments && + Objects.equals(dataSource, that.dataSource) && + Objects.equals(intervals, that.intervals) && + Objects.equals(segments, that.segments) && + Objects.equals(filter, that.filter) && + Objects.equals(dimensions, that.dimensions) && + Objects.equals(metrics, that.metrics) && + Objects.equals(transformSpec, that.transformSpec); } @Override public int hashCode() { - int result = dataSource.hashCode(); - result = 31 * result + intervals.hashCode(); - result = 31 * result + (segments != null ? segments.hashCode() : 0); - result = 31 * result + (filter != null ? filter.hashCode() : 0); - result = 31 * result + (dimensions != null ? dimensions.hashCode() : 0); - result = 31 * result + (metrics != null ? metrics.hashCode() : 0); - result = 31 * result + (ignoreWhenNoSegments ? 1 : 0); - return result; + return Objects.hash( + dataSource, + intervals, + segments, + filter, + dimensions, + metrics, + ignoreWhenNoSegments, + transformSpec + ); } @Override @@ -236,6 +257,7 @@ public String toString() ", dimensions=" + dimensions + ", metrics=" + metrics + ", ignoreWhenNoSegments=" + ignoreWhenNoSegments + + ", transformSpec=" + transformSpec + '}'; } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java index c703b0380f2a..b4856a925fca 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceInputFormat.java @@ -24,13 +24,11 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.collect.Lists; - import io.druid.data.input.InputRow; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.JobHelper; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -61,6 +59,7 @@ public class DatasourceInputFormat extends InputFormat public static final String CONF_INPUT_SEGMENTS = "druid.segments"; public static final String CONF_DRUID_SCHEMA = "druid.datasource.schema"; + public static final String CONF_TRANSFORM_SPEC = "druid.datasource.transformSpec"; public static final String CONF_MAX_SPLIT_SIZE = "druid.datasource.split.max.size"; @Override diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java index 94129d72030e..7f364a36782d 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/hadoop/DatasourceRecordReader.java @@ -105,11 +105,9 @@ public WindowedStorageAdapter apply(WindowedDataSegment segment) } ); - final HadoopDruidIndexerConfig config = HadoopDruidIndexerConfig.fromConfiguration(context.getConfiguration()); - firehose = new IngestSegmentFirehose( adapters, - config.getSchema().getDataSchema().getTransformSpec(), + spec.getTransformSpec(), spec.getDimensions(), spec.getMetrics(), spec.getFilter() diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/path/DatasourcePathSpec.java b/indexing-hadoop/src/main/java/io/druid/indexer/path/DatasourcePathSpec.java index 15711eabb83b..e89ca3c0f0f9 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/path/DatasourcePathSpec.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/path/DatasourcePathSpec.java @@ -28,7 +28,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.indexer.hadoop.DatasourceIngestionSpec; import io.druid.indexer.hadoop.DatasourceInputFormat; @@ -158,6 +157,11 @@ public Iterable apply(WindowedDataSegment dataSegment) updatedIngestionSpec = updatedIngestionSpec.withQueryGranularity(config.getGranularitySpec().getQueryGranularity()); + // propagate in the transformSpec from the overall job config + updatedIngestionSpec = updatedIngestionSpec.withTransformSpec( + config.getSchema().getDataSchema().getTransformSpec() + ); + job.getConfiguration().set(DatasourceInputFormat.CONF_DRUID_SCHEMA, mapper.writeValueAsString(updatedIngestionSpec)); job.getConfiguration().set(DatasourceInputFormat.CONF_INPUT_SEGMENTS, mapper.writeValueAsString(segments)); job.getConfiguration().set(DatasourceInputFormat.CONF_MAX_SPLIT_SIZE, String.valueOf(maxSplitSize)); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java index d019689262c2..e9d92e55c0ee 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java @@ -93,7 +93,7 @@ public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePat PathSpec pathSpec = new DatasourcePathSpec( jsonMapper, null, - new DatasourceIngestionSpec(testDatasource, testDatasourceInterval, null, null, null, null, null, false), + new DatasourceIngestionSpec(testDatasource, testDatasourceInterval, null, null, null, null, null, false, null), null ); HadoopDruidIndexerConfig config = testRunUpdateSegmentListIfDatasourcePathSpecIsUsed( @@ -120,7 +120,8 @@ public void testupdateSegmentListIfDatasourcePathSpecWithMatchingUserSegments() null, null, null, - false + false, + null ), null ); @@ -148,7 +149,8 @@ public void testupdateSegmentListThrowsExceptionWithUserSegmentsMismatch() throw null, null, null, - false + false, + null ), null ); @@ -173,7 +175,8 @@ public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithJustDatasourcePat null, null, null, - false + false, + null ), null ); @@ -204,7 +207,8 @@ public void testupdateSegmentListIfDatasourcePathSpecIsUsedWithMultiplePathSpec( null, null, null, - false + false, + null ), null ) diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java index adbeeea44861..cee6f6833222 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceIngestionSpecTest.java @@ -51,7 +51,8 @@ public void testSingleIntervalSerde() throws Exception new SelectorDimFilter("dim", "value", null), Lists.newArrayList("d1", "d2"), Lists.newArrayList("m1", "m2", "m3"), - false + false, + null ); DatasourceIngestionSpec actual = MAPPER.readValue(MAPPER.writeValueAsString(expected), DatasourceIngestionSpec.class); @@ -85,7 +86,8 @@ public void testMultiIntervalSerde() throws Exception null, null, null, - false + false, + null ); Assert.assertEquals(expected, actual); @@ -133,7 +135,8 @@ public void testMultiIntervalSerde() throws Exception new SelectorDimFilter("dim", "value", null), Lists.newArrayList("d1", "d2"), Lists.newArrayList("m1", "m2", "m3"), - true + true, + null ); actual = MAPPER.readValue( @@ -153,7 +156,7 @@ public void testOldJsonDeserialization() throws Exception DatasourceIngestionSpec actual = MAPPER.readValue(jsonStr, DatasourceIngestionSpec.class); Assert.assertEquals( - new DatasourceIngestionSpec("test", Intervals.of("2014/2015"), null, null, null, null, null, false), + new DatasourceIngestionSpec("test", Intervals.of("2014/2015"), null, null, null, null, null, false, null), actual ); } diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java index 13a25d1cd1c7..237ecfba03f5 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/hadoop/DatasourceRecordReaderTest.java @@ -69,7 +69,8 @@ public void testSanity() throws Exception null, segment.getDimensions(), segment.getMetrics(), - false + false, + null ) ) ); diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java index a13e7dd64861..ab1b0780a9ed 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/path/DatasourcePathSpecTest.java @@ -79,7 +79,8 @@ public DatasourcePathSpecTest() null, null, null, - false + false, + null ); segments = ImmutableList.of( From 07478d7dfdc9e8f4ee9a9f181f95be63db221c5d Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 17 Oct 2017 10:06:13 -0700 Subject: [PATCH 15/23] Fix test. --- .../io/druid/indexing/common/task/RealtimeIndexTaskTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 6d15801f7acb..a5b3c95a1df5 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -489,7 +489,7 @@ public void testReportParseExceptionsOnBadMetric() throws Exception CoreMatchers.allOf( CoreMatchers.instanceOf(ParseException.class), ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("Unable to parse metrics[met1], value[foo]") + CoreMatchers.containsString("Unable to parse value[foo] for field[met1]") ) ) ) From 5e81b6ea4cf0212a79cb9234f616089fb5140b01 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Oct 2017 13:05:05 -0700 Subject: [PATCH 16/23] CR comments. --- .../java/io/druid/indexer/HadoopDruidIndexerMapper.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java index 78d9e31d3fcf..a90bd38ac245 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/HadoopDruidIndexerMapper.java @@ -76,18 +76,14 @@ protected void map( if (reportParseExceptions) { throw e; } - if (log.isDebugEnabled()) { - log.debug(e, "Ignoring invalid row [%s] due to parsing error", value.toString()); - } + log.debug(e, "Ignoring invalid row [%s] due to parsing error", value); context.getCounter(HadoopDruidIndexerConfig.IndexJobCounters.INVALID_ROW_COUNTER).increment(1); return; // we're ignoring this invalid row } if (inputRow == null) { // Throw away null rows from the parser. - if (log.isDebugEnabled()) { - log.debug("Throwing away row [%s]", value.toString()); - } + log.debug("Throwing away row [%s]", value); return; } From bf5d60ae935e3d26a188fd74b9dce9d21235ebea Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Oct 2017 13:48:33 -0700 Subject: [PATCH 17/23] Remove unused method. --- .../main/java/io/druid/segment/indexing/Transformer.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/server/src/main/java/io/druid/segment/indexing/Transformer.java b/server/src/main/java/io/druid/segment/indexing/Transformer.java index 78f273e914e9..8a18f898c43e 100644 --- a/server/src/main/java/io/druid/segment/indexing/Transformer.java +++ b/server/src/main/java/io/druid/segment/indexing/Transformer.java @@ -189,14 +189,5 @@ public int compareTo(final Row o) { return row.compareTo(o); } - - private Object getValueFromRow(final String column) - { - if (column.equals(Column.TIME_COLUMN_NAME)) { - return row.getTimestampFromEpoch(); - } else { - return row.getRaw(column); - } - } } } From 20e5a935721a50e25b7d70ebaf0656d2026b3856 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Oct 2017 13:53:07 -0700 Subject: [PATCH 18/23] Add javadocs. --- .../java/io/druid/segment/indexing/Transform.java | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/server/src/main/java/io/druid/segment/indexing/Transform.java b/server/src/main/java/io/druid/segment/indexing/Transform.java index cb8db5b3c9f6..41440a97b0f5 100644 --- a/server/src/main/java/io/druid/segment/indexing/Transform.java +++ b/server/src/main/java/io/druid/segment/indexing/Transform.java @@ -22,13 +22,24 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +/** + * A row transform that is part of a {@link TransformSpec}. + */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @JsonSubTypes.Type(name = "expression", value = ExpressionTransform.class) }) public interface Transform { + /** + * Returns the column name for this transform. DimensionSpecs, AggregatorFactories, etc, can refer to this transform + * by its name. + */ String getName(); + /** + * Returns the function for this transform. The RowFunction takes an entire row as input and returns a column value + * as output. + */ RowFunction getRowFunction(); } From c4f8e377b2a28689036eea914a3b7f1097deb440 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 20 Oct 2017 14:16:07 -0700 Subject: [PATCH 19/23] More javadocs, and always decorate. --- .../io/druid/segment/indexing/Transform.java | 14 +++++++--- .../druid/segment/indexing/TransformSpec.java | 16 +++++++++--- .../segment/indexing/TransformSpecTest.java | 26 +++++++++++++++++++ 3 files changed, 49 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/io/druid/segment/indexing/Transform.java b/server/src/main/java/io/druid/segment/indexing/Transform.java index 41440a97b0f5..9c8253f066b8 100644 --- a/server/src/main/java/io/druid/segment/indexing/Transform.java +++ b/server/src/main/java/io/druid/segment/indexing/Transform.java @@ -23,7 +23,16 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; /** - * A row transform that is part of a {@link TransformSpec}. + * A row transform that is part of a {@link TransformSpec}. Transforms allow adding new fields to input rows. Each + * one has a "name" (the name of the new field) which can be referred to by DimensionSpecs, AggregatorFactories, etc. + * Each also has a "row function", which produces values for this new field based on looking at the entire input row. + * + * If a transform has the same name as a field in an input row, then it will shadow the original field. Transforms + * that shadow fields may still refer to the fields they shadow. This can be used to transform a field "in-place". + * + * Transforms do have some limitations. They can only refer to fields present in the actual input rows; in particular, + * they cannot refer to other transforms. And they cannot remove fields, only add them. However, they can shadow a + * field with another field containing all nulls, which will act similarly to removing the field. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @@ -32,8 +41,7 @@ public interface Transform { /** - * Returns the column name for this transform. DimensionSpecs, AggregatorFactories, etc, can refer to this transform - * by its name. + * Returns the field name for this transform. */ String getName(); diff --git a/server/src/main/java/io/druid/segment/indexing/TransformSpec.java b/server/src/main/java/io/druid/segment/indexing/TransformSpec.java index dddbcd37a047..ff441834a07d 100644 --- a/server/src/main/java/io/druid/segment/indexing/TransformSpec.java +++ b/server/src/main/java/io/druid/segment/indexing/TransformSpec.java @@ -33,6 +33,13 @@ import java.util.Objects; import java.util.Set; +/** + * Specifies how input rows should be filtered and transforms. There are two parts: a "filter" (which can filter out + * input rows) and "transforms" (which can add fields to input rows). Filters may refer to fields generated by + * a transform. + * + * See {@link Transform} for details on how each transform works. + */ public class TransformSpec { public static final TransformSpec NONE = new TransformSpec(null, null); @@ -69,7 +76,7 @@ public static TransformSpec fromInputRowParser(final InputRowParser parse } else if (parser instanceof TransformingStringInputRowParser) { return ((TransformingStringInputRowParser) parser).getTransformSpec(); } else { - return TransformSpec.NONE; + throw new ISE("Parser was not decorated, but should have been"); } } @@ -88,9 +95,10 @@ public List getTransforms() public InputRowParser decorate(final InputRowParser parser) { - if (filter == null && transforms.isEmpty()) { - return parser; - } else if (parser instanceof StringInputRowParser) { + // Always decorates, even if the transformSpec is a no-op. This is so fromInputRowParser can insist that the + // parser is a transforming parser, and possibly help detect coding errors where someone forgot to call "decorate". + + if (parser instanceof StringInputRowParser) { // Hack to support the fact that some callers use special methods in StringInputRowParser, such as // parse(String) and startFileFromBeginning. return (InputRowParser) new TransformingStringInputRowParser( diff --git a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java index b9a7cede7113..72bd851ae971 100644 --- a/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java +++ b/server/src/test/java/io/druid/segment/indexing/TransformSpecTest.java @@ -93,9 +93,35 @@ public void testTransforms() Assert.assertEquals(5L, row.getMetric("g").longValue()); } + @Test + public void testTransformOverwriteField() + { + // Transforms are allowed to overwrite fields, and to refer to the fields they overwrite; double-check this. + + final TransformSpec transformSpec = new TransformSpec( + null, + ImmutableList.of( + new ExpressionTransform("x", "concat(x,y)", TestExprMacroTable.INSTANCE) + ) + ); + + final InputRowParser> parser = transformSpec.decorate(PARSER); + final InputRow row = parser.parse(ROW1); + + Assert.assertNotNull(row); + Assert.assertEquals(DateTimes.of("2000-01-01").getMillis(), row.getTimestampFromEpoch()); + Assert.assertEquals(DateTimes.of("2000-01-01"), row.getTimestamp()); + Assert.assertEquals(ImmutableList.of("f", "x", "y"), row.getDimensions()); + Assert.assertEquals(ImmutableList.of("foobar"), row.getDimension("x")); + Assert.assertEquals(3.0, row.getMetric("b").doubleValue(), 0); + Assert.assertNull(row.getRaw("f")); + } + @Test public void testFilterOnTransforms() { + // Filters are allowed to refer to transformed fields; double-check this. + final TransformSpec transformSpec = new TransformSpec( new AndDimFilter( ImmutableList.of( From bdd7ff83b8186d5716b60a5d0413f78a2b72aaf0 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 26 Oct 2017 12:53:58 -0700 Subject: [PATCH 20/23] Fix bug in TransformingStringInputRowParser. --- .../druid/data/input/impl/DimensionsSpec.java | 12 ++++++- .../druid/data/input/impl/JSONParseSpec.java | 35 +++++++++++++++++++ .../indexing/TransformingInputRowParser.java | 1 + .../TransformingStringInputRowParser.java | 2 +- .../segment/indexing/DataSchemaTest.java | 2 +- 5 files changed, 49 insertions(+), 3 deletions(-) diff --git a/api/src/main/java/io/druid/data/input/impl/DimensionsSpec.java b/api/src/main/java/io/druid/data/input/impl/DimensionsSpec.java index 3d48a95abe71..12e0c3029f05 100644 --- a/api/src/main/java/io/druid/data/input/impl/DimensionsSpec.java +++ b/api/src/main/java/io/druid/data/input/impl/DimensionsSpec.java @@ -121,7 +121,8 @@ public Set getDimensionExclusions() return dimensionExclusions; } - @Deprecated @JsonIgnore + @Deprecated + @JsonIgnore public List getSpatialDimensions() { Iterable filteredList = Iterables.filter( @@ -244,4 +245,13 @@ public int hashCode() result = 31 * result + dimensionExclusions.hashCode(); return result; } + + @Override + public String toString() + { + return "DimensionsSpec{" + + "dimensions=" + dimensions + + ", dimensionExclusions=" + dimensionExclusions + + '}'; + } } diff --git a/api/src/main/java/io/druid/data/input/impl/JSONParseSpec.java b/api/src/main/java/io/druid/data/input/impl/JSONParseSpec.java index 357cbb16a614..ce33d41ed481 100644 --- a/api/src/main/java/io/druid/data/input/impl/JSONParseSpec.java +++ b/api/src/main/java/io/druid/data/input/impl/JSONParseSpec.java @@ -30,6 +30,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; /** */ @@ -97,4 +98,38 @@ public Map getFeatureSpec() { return featureSpec; } + + @Override + public boolean equals(final Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + final JSONParseSpec that = (JSONParseSpec) o; + return Objects.equals(flattenSpec, that.flattenSpec) && + Objects.equals(featureSpec, that.featureSpec); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), flattenSpec, featureSpec); + } + + @Override + public String toString() + { + return "JSONParseSpec{" + + "timestampSpec=" + getTimestampSpec() + + ", dimensionsSpec=" + getDimensionsSpec() + + ", flattenSpec=" + flattenSpec + + ", featureSpec=" + featureSpec + + '}'; + } } diff --git a/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java b/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java index 6fbe05bf1e01..96151b585f6e 100644 --- a/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java +++ b/server/src/main/java/io/druid/segment/indexing/TransformingInputRowParser.java @@ -49,6 +49,7 @@ public ParseSpec getParseSpec() } @Override + @SuppressWarnings("unchecked") public InputRowParser withParseSpec(final ParseSpec parseSpec) { return new TransformingInputRowParser<>(parser.withParseSpec(parseSpec), transformSpec); diff --git a/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java b/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java index 9a346298fbe8..8a241969136c 100644 --- a/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java +++ b/server/src/main/java/io/druid/segment/indexing/TransformingStringInputRowParser.java @@ -58,7 +58,7 @@ public InputRow parse(@Nullable final String input) @Override public StringInputRowParser withParseSpec(final ParseSpec parseSpec) { - return new TransformingStringInputRowParser(getParseSpec(), getEncoding(), transformSpec); + return new TransformingStringInputRowParser(parseSpec, getEncoding(), transformSpec); } public TransformSpec getTransformSpec() diff --git a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java index abca29154ae8..730a27ba8e17 100644 --- a/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/io/druid/segment/indexing/DataSchemaTest.java @@ -322,7 +322,7 @@ public void testSerde() throws Exception null ) ); - Assert.assertEquals( + Assert.assertArrayEquals( actual.getAggregators(), new AggregatorFactory[]{ new DoubleSumAggregatorFactory("metric1", "col1") From f90482d3396f742469dcf09d98911b002df3fac3 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Thu, 26 Oct 2017 14:50:06 -0700 Subject: [PATCH 21/23] Fix bad merge. --- .../druid/indexing/kafka/KafkaIndexTask.java | 41 +++++++++++-------- 1 file changed, 23 insertions(+), 18 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 625c1a4eb70e..02c11333d678 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -80,8 +80,8 @@ import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.server.security.Access; import io.druid.server.security.Action; -import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.AuthorizationUtils; +import io.druid.server.security.AuthorizerMapper; import io.druid.server.security.ForbiddenException; import io.druid.server.security.Resource; import io.druid.server.security.ResourceAction; @@ -491,21 +491,6 @@ public void run() fireDepartmentMetrics.incrementProcessed(); } else { - if (log.isDebugEnabled()) { - if (beforeMinimumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMinimumMessageTime().get() - ); - } else if (afterMaximumMessageTime) { - log.debug( - "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", - row.getTimestamp(), - ioConfig.getMaximumMessageTime().get() - ); - } - } fireDepartmentMetrics.incrementThrownAway(); } } @@ -1225,8 +1210,28 @@ private void sendResetRequestAndWait(Map outOfRangePartiti private boolean withinMinMaxRecordTime(final InputRow row) { - final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); - final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + final boolean beforeMinimumMessageTime = ioConfig.getMinimumMessageTime().isPresent() + && ioConfig.getMinimumMessageTime().get().isAfter(row.getTimestamp()); + + final boolean afterMaximumMessageTime = ioConfig.getMaximumMessageTime().isPresent() + && ioConfig.getMaximumMessageTime().get().isBefore(row.getTimestamp()); + + if (log.isDebugEnabled()) { + if (beforeMinimumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is before MinimumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMinimumMessageTime().get() + ); + } else if (afterMaximumMessageTime) { + log.debug( + "CurrentTimeStamp[%s] is after MaximumMessageTime[%s]", + row.getTimestamp(), + ioConfig.getMaximumMessageTime().get() + ); + } + } + return !beforeMinimumMessageTime && !afterMaximumMessageTime; } } From 8ab2dc7edf8635390b79f023faccaf602a21b3c6 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Fri, 27 Oct 2017 10:58:03 -0700 Subject: [PATCH 22/23] Fix ISFF tests. --- .../IngestSegmentFirehoseFactoryTest.java | 4 +++- ...estSegmentFirehoseFactoryTimelineTest.java | 21 ++++++++++++------- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 94509c085318..bdda4aae5ca6 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -400,7 +400,9 @@ public IngestSegmentFirehoseFactoryTest( ) { this.factory = factory; - this.rowParser = rowParser; + + // Must decorate the parser, since IngestSegmentFirehoseFactory will undecorate it. + this.rowParser = TransformSpec.NONE.decorate(rowParser); } private static final Logger log = new Logger(IngestSegmentFirehoseFactoryTest.class); diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 822301459e33..58d21ee3aa8f 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -59,6 +59,7 @@ import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.incremental.IndexSizeExceededException; +import io.druid.segment.indexing.TransformSpec; import io.druid.segment.loading.SegmentLoaderConfig; import io.druid.segment.loading.SegmentLoaderLocalCacheManager; import io.druid.segment.loading.StorageLocationConfig; @@ -91,16 +92,20 @@ public class IngestSegmentFirehoseFactoryTimelineTest private static final String TIME_COLUMN = "t"; private static final String[] DIMENSIONS = new String[]{"d1"}; private static final String[] METRICS = new String[]{"m1"}; - private static final InputRowParser> ROW_PARSER = new MapInputRowParser( - new JSONParseSpec( - new TimestampSpec(TIME_COLUMN, "auto", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS)), + + // Must decorate the parser, since IngestSegmentFirehoseFactory will undecorate it. + private static final InputRowParser> ROW_PARSER = TransformSpec.NONE.decorate( + new MapInputRowParser( + new JSONParseSpec( + new TimestampSpec(TIME_COLUMN, "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS)), + null, + null + ), null, null - ), - null, - null + ) ) ); From aa4c93087ed76b47585a7a973b4797484769583b Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Mon, 30 Oct 2017 15:05:06 -0700 Subject: [PATCH 23/23] Fix DORC test. --- .../java/io/druid/data/input/orc/DruidOrcInputFormatTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/DruidOrcInputFormatTest.java b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/DruidOrcInputFormatTest.java index b0cb778c3024..9736b10b40d0 100644 --- a/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/DruidOrcInputFormatTest.java +++ b/extensions-contrib/orc-extensions/src/test/java/io/druid/data/input/orc/DruidOrcInputFormatTest.java @@ -19,6 +19,7 @@ package io.druid.data.input.orc; import io.druid.data.input.MapBasedInputRow; +import io.druid.data.input.impl.InputRowParser; import io.druid.indexer.HadoopDruidIndexerConfig; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.StringUtils; @@ -90,7 +91,7 @@ public void testRead() throws IOException, InterruptedException TaskAttemptContext context = new TaskAttemptContextImpl(job.getConfiguration(), new TaskAttemptID()); RecordReader reader = inputFormat.createRecordReader(split, context); - OrcHadoopInputRowParser parser = (OrcHadoopInputRowParser) config.getParser(); + InputRowParser parser = (InputRowParser) config.getParser(); reader.initialize(split, context);