diff --git a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java index b3fda99b222e..161e80569521 100644 --- a/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java +++ b/extensions-contrib/kubernetes-overlord-extensions/src/test/java/org/apache/druid/k8s/overlord/common/K8sTestUtils.java @@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -66,18 +65,19 @@ public static Task getTask() null, null, new IndexTask.IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexTask.IndexIOConfig( new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index 01039375259e..b23af62f6309 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -48,7 +48,6 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.realtime.ChatHandlerProvider; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -211,14 +210,13 @@ public HadoopIndexTask createTask(Interval interval, String version, List runningTasks = runningTasksPair.lhs; Map runningVersion = runningTasksPair.rhs; - DataSchema dataSchema = new DataSchema( - "test_datasource", - null, - null, - null, - TransformSpec.NONE, - objectMapper - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_datasource") + .withObjectMapper(objectMapper) + .build(); HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(new HashMap<>(), null, null); HadoopIngestionSpec spec = new HadoopIngestionSpec(dataSchema, hadoopIOConfig, null); HadoopIndexTask task1 = new HadoopIndexTask( diff --git a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java index 5f31d70b75dd..e48925ee3b41 100644 --- a/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java +++ b/extensions-contrib/rabbit-stream-indexing-service/src/test/java/org/apache/druid/indexing/rabbitstream/supervisor/RabbitStreamSupervisorTest.java @@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -44,7 +43,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -102,16 +100,19 @@ private static DataSchema getDataSchema(String dataSource) dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - dataSource, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[] {new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of()), - null); + return DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } @BeforeClass diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java index eb874dbea27d..fe60d3eda442 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactory.java @@ -278,6 +278,26 @@ public boolean equals(Object o) && stringEncoding == that.stringEncoding; } + @Nullable + @Override + public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated) + { + if (this == preAggregated) { + return getCombiningFactory(); + } + if (getClass() != preAggregated.getClass()) { + return null; + } + HllSketchAggregatorFactory that = (HllSketchAggregatorFactory) preAggregated; + if (lgK == that.lgK && tgtHllType == that.tgtHllType && stringEncoding == that.stringEncoding && Objects.equals( + fieldName, + that.fieldName + )) { + return getCombiningFactory(); + } + return null; + } + @Override public int hashCode() { diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java index dcc12f5c3847..12bbccce9d02 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchAggregatorFactoryTest.java @@ -231,6 +231,42 @@ public void testEqualsOtherMatches() Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey()); } + @Test + public void testCanSubstitute() + { + HllSketchBuildAggregatorFactory factory = new HllSketchBuildAggregatorFactory( + NAME, + FIELD_NAME, + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + true, + true + ); + HllSketchBuildAggregatorFactory other = new HllSketchBuildAggregatorFactory( + "other name", + FIELD_NAME, + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + false, + false + ); + + HllSketchBuildAggregatorFactory incompatible = new HllSketchBuildAggregatorFactory( + NAME, + "different field", + LG_K, + TGT_HLL_TYPE, + STRING_ENCODING, + false, + false + ); + Assert.assertNotNull(other.substituteCombiningFactory(factory)); + Assert.assertNotNull(factory.substituteCombiningFactory(other)); + Assert.assertNull(factory.substituteCombiningFactory(incompatible)); + } + @Test public void testToString() { diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index f8c6b23aae90..d2876f9b8a6a 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -92,7 +92,6 @@ import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; @@ -1262,28 +1261,27 @@ public void testKafkaRecordEntityInputFormat() throws Exception final KafkaIndexTask task = createTask( null, - new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat"), - new StringDimensionSchema("kafka.topic"), - new LongDimensionSchema("kafka.offset"), - new StringDimensionSchema("kafka.header.encoding") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ), + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat"), + new StringDimensionSchema("kafka.topic"), + new LongDimensionSchema("kafka.offset"), + new StringDimensionSchema("kafka.header.encoding") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(), new KafkaIndexTaskIOConfig( 0, "sequence0", @@ -1337,26 +1335,25 @@ public void testKafkaInputFormat() throws Exception final KafkaIndexTask task = createTask( null, - new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat"), - new StringDimensionSchema("kafka.testheader.encoding") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ), + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat"), + new StringDimensionSchema("kafka.testheader.encoding") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(), new KafkaIndexTaskIOConfig( 0, "sequence0", @@ -2887,16 +2884,7 @@ private KafkaIndexTask createTask( private static DataSchema cloneDataSchema(final DataSchema dataSchema) { - return new DataSchema( - dataSchema.getDataSource(), - dataSchema.getTimestampSpec(), - dataSchema.getDimensionsSpec(), - dataSchema.getAggregators(), - dataSchema.getGranularitySpec(), - dataSchema.getTransformSpec(), - dataSchema.getParserMap(), - OBJECT_MAPPER - ); + return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build(); } @Override diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java index 0a0b64396a66..9cdc0ac0edcd 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaSamplerSpecTest.java @@ -46,7 +46,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.TestHelper; @@ -81,45 +80,30 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); private static final String TOPIC = "sampling"; - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); - - private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP = new DataSchema( - "test_ds", - new TimestampSpec("kafka.timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(); + + private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP = + DataSchema.builder(DATA_SCHEMA) + .withTimestamp(new TimestampSpec("kafka.timestamp", "iso", null)) + .build(); private static TestingCluster zkServer; private static TestBroker kafkaServer; @@ -364,17 +348,18 @@ public void testWithInputRowParser() throws IOException ); InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); - DataSchema dataSchema = new DataSchema( - "test_ds", - objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - objectMapper - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(objectMapper) + .build(); KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec( null, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 86275d10e318..90d7a3a824ab 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -30,7 +30,6 @@ import org.apache.curator.test.TestingCluster; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -83,7 +82,6 @@ import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.ParseExceptionReport; @@ -5128,18 +5126,19 @@ private static DataSchema getDataSchema(String dataSource) dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - dataSource, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } private KafkaIndexTask createKafkaIndexTask( diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java index 60d8f686a28c..7fa5e2158f8b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskSerdeTest.java @@ -50,7 +50,8 @@ public class KinesisIndexTaskSerdeTest { - private static final DataSchema DATA_SCHEMA = new DataSchema("dataSource", null, null, null, null, null, null, null); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder().withDataSource("dataSource").build(); private static final KinesisIndexTaskTuningConfig TUNING_CONFIG = new KinesisIndexTaskTuningConfig( null, null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index 510eaa797e07..b1fdc12338da 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2411,16 +2411,7 @@ private KinesisIndexTask createTask( private static DataSchema cloneDataSchema(final DataSchema dataSchema) { - return new DataSchema( - dataSchema.getDataSource(), - dataSchema.getTimestampSpec(), - dataSchema.getDimensionsSpec(), - dataSchema.getAggregators(), - dataSchema.getGranularitySpec(), - dataSchema.getTransformSpec(), - dataSchema.getParserMap(), - OBJECT_MAPPER - ); + return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build(); } @Override diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java index 63144c6a9353..102e2d8929f2 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisSamplerSpecTest.java @@ -49,7 +49,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -75,25 +74,25 @@ public class KinesisSamplerSpecTest extends EasyMockSupport { private static final String STREAM = "sampling"; private static final String SHARD_ID = "1"; - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null) + ) + .build(); static { NullHandling.initializeForTests(); @@ -192,17 +191,18 @@ public void testSampleWithInputRowParser() throws IOException, InterruptedExcept ); InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); - DataSchema dataSchema = new DataSchema( - "test_ds", - objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - objectMapper - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(objectMapper) + .build(); KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec( null, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java index e6ed27c9cecc..efbcd33deb9f 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java @@ -28,7 +28,6 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -79,7 +78,6 @@ import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -5479,18 +5477,19 @@ private static DataSchema getDataSchema(String dataSource) dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - dataSource, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(dataSource) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java index 09d79534337c..b37a29f53e20 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/destination/SegmentGenerationUtils.java @@ -53,14 +53,12 @@ import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashSet; import java.util.List; @@ -96,14 +94,13 @@ public static DataSchema makeDataSchemaForIngestion( destination.getDimensionSchemas() ); - return new DataSchema( - destination.getDataSource(), - new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), - dimensionsAndAggregators.lhs, - dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]), - makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper), - new TransformSpec(null, Collections.emptyList()) - ); + return DataSchema.builder() + .withDataSource(destination.getDataSource()) + .withTimestamp(new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null)) + .withDimensions(dimensionsAndAggregators.lhs) + .withAggregators(dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0])) + .withGranularity(makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper)) + .build(); } private static GranularitySpec makeGranularitySpecForIngestion( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 4088d5cecb10..15b12be15753 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -259,19 +259,21 @@ public void testMSQControllerTaskSpecWithScanIsValid() throws JsonProcessingExce null ); - DataSchema dataSchema = new DataSchema( - DATA_SOURCE, - new TimestampSpec(TIMESTAMP_COLUMN, null, null), - new DimensionsSpec(DIMENSIONS), - new AggregatorFactory[]{}, - new UniformGranularitySpec( - SEGMENT_GRANULARITY.getDefaultGranularity(), - null, - false, - Collections.singletonList(COMPACTION_INTERVAL) - ), - new TransformSpec(dimFilter, Collections.emptyList()) - ); + DataSchema dataSchema = + DataSchema.builder() + .withDataSource(DATA_SOURCE) + .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null)) + .withDimensions(DIMENSIONS) + .withGranularity( + new UniformGranularitySpec( + SEGMENT_GRANULARITY.getDefaultGranularity(), + null, + false, + Collections.singletonList(COMPACTION_INTERVAL) + ) + ) + .withTransform(new TransformSpec(dimFilter, Collections.emptyList())) + .build(); List msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java index d0410897a07e..26a360e716d3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/shuffle/output/ChannelStageOutputReaderTest.java @@ -296,7 +296,7 @@ public void setUp() frameReader = FrameReader.create(adapter.getRowSignature()); frameList = FrameSequenceBuilder.fromCursorFactory(adapter) .frameType(FrameType.ROW_BASED) - .maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING)) + .maxRowsPerFrame(IntMath.divide(index.numRows(), MAX_FRAMES, RoundingMode.CEILING)) .frames() .toList(); } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java index 1e4f62ca6e1f..41dae9c9e65e 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/BatchDeltaIngestionTest.java @@ -433,30 +433,33 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( { HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), - null, - ImmutableList.of("timestamp", "host", "host2", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - aggregators != null ? aggregators : new AggregatorFactory[]{ - new LongSumAggregatorFactory("visited_sum", "visited_num"), - new HyperUniquesAggregatorFactory("unique_hosts", "host2") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)), - null, - MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap(MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), + null, + ImmutableList.of("timestamp", "host", "host2", "visited_num"), + false, + 0 + ), + null + ), + Map.class + )) + .withAggregators(aggregators != null ? aggregators : new AggregatorFactory[]{ + new LongSumAggregatorFactory("visited_sum", "visited_num"), + new HyperUniquesAggregatorFactory("unique_hosts", "host2") + }) + .withGranularity(new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(INTERVAL_FULL) + )) + .withObjectMapper(MAPPER) + .build(), new HadoopIOConfig( inputSpec, null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java index 24a8ee0ef7eb..dd22a95083ce 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineHashedPartitionsJobTest.java @@ -32,7 +32,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -158,46 +157,45 @@ public DetermineHashedPartitionsJobTest( } HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec( - new DataSchema( - "test_schema", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new DelimitedParseSpec( - new TimestampSpec("ts", null, null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of( - "market", - "quality", - "placement", - "placementish" - )) - ), - "\t", - null, - Arrays.asList( - "ts", - "market", - "quality", - "placement", - "placementish", - "index" - ), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new DoubleSumAggregatorFactory("index", "index")}, - new UniformGranularitySpec( - segmentGranularity, - Granularities.NONE, - intervals - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("test_schema") + .withParserMap(HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new DelimitedParseSpec( + new TimestampSpec("ts", null, null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of( + "market", + "quality", + "placement", + "placementish" + )) + ), + "\t", + null, + Arrays.asList( + "ts", + "market", + "quality", + "placement", + "placementish", + "index" + ), + false, + 0 + ), + null + ), + Map.class + )) + .withAggregators(new DoubleSumAggregatorFactory("index", "index")) + .withGranularity(new UniformGranularitySpec( + segmentGranularity, + Granularities.NONE, + intervals + )) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java index a3c98f29565b..bfd28d2cfca0 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DeterminePartitionsJobTest.java @@ -29,7 +29,6 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -280,33 +279,36 @@ public DeterminePartitionsJobTest( config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) - ), - null, - ImmutableList.of("timestamp", "host", "country", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of(interval)) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) + ), + null, + ImmutableList.of("timestamp", "host", "country", "visited_num"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of(interval)) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java index e79d066ab55c..3ff525c8b433 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/DetermineRangePartitionsJobTest.java @@ -29,7 +29,6 @@ import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -328,35 +327,36 @@ public DetermineRangePartitionsJobTest( config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - null, - null, - new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of(interval)) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) - ), - null, - ImmutableList.of("timestamp", "host", "country", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of(interval)) + ) + ) + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) + ), + null, + ImmutableList.of("timestamp", "host", "country", "visited_num"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java index 8aead05d625b..ed47d180b432 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerConfigTest.java @@ -37,7 +37,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; @@ -217,18 +216,18 @@ public void testGetTargetPartitionSizeWithSingleDimensionPartitionsMaxRowsPerSeg private static class HadoopIngestionSpecBuilder { - private static final DataSchema DATA_SCHEMA = new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.MINUTE, - Granularities.MINUTE, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - HadoopDruidIndexerConfigTest.JSON_MAPPER - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.MINUTE, + Granularities.MINUTE, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfigTest.JSON_MAPPER) + .build(); private static final HadoopIOConfig HADOOP_IO_CONFIG = new HadoopIOConfig( ImmutableMap.of("paths", "bar", "type", "static"), diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java index db20ed8a1847..da57b8ccf4a3 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopDruidIndexerMapperTest.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.TestHelper; @@ -58,27 +57,29 @@ public class HadoopDruidIndexerMapperTest { private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); - 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")) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - null - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - JSON_MAPPER - ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + JSON_MAPPER.convertValue( + new HadoopyStringInputRowParser( + new JSONParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + null + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ) + ) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(JSON_MAPPER) + .build(); private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( JSON_MAPPER.convertValue( diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java index 6402721e73c6..afcfb4023595 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java @@ -34,7 +34,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.timeline.DataSegment; @@ -274,18 +273,17 @@ private HadoopDruidIndexerConfig testRunUpdateSegmentListIfDatasourcePathSpecIsU throws Exception { HadoopIngestionSpec spec = new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), new HadoopIOConfig( jsonMapper.convertValue(datasourcePathSpec, Map.class), null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java index 385c28ff0fb0..37cdbb7300d7 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorCombinerTest.java @@ -64,30 +64,33 @@ public void setUp() throws Exception { HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new TimeAndDimsParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords"))) - ), - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("visited_sum", "visited"), - new HyperUniquesAggregatorFactory("unique_hosts", "host") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of("2010/2011")) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new TimeAndDimsParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords"))) + ), + null + ), + Map.class + ) + ) + .withAggregators( + new LongSumAggregatorFactory("visited_sum", "visited"), + new HyperUniquesAggregatorFactory("unique_hosts", "host") + ) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of("2010/2011")) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java index e14ade454f4c..241746ca58d6 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/IndexGeneratorJobTest.java @@ -506,17 +506,19 @@ public void setUp() throws Exception config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - datasourceName, - mapper.convertValue( - inputRowParser, - Map.class - ), - aggs, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)), - null, - mapper - ), + DataSchema.builder() + .withDataSource(datasourceName) + .withParserMap(mapper.convertValue(inputRowParser, Map.class)) + .withAggregators(aggs) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(interval) + ) + ) + .withObjectMapper(mapper) + .build(), new HadoopIOConfig( ImmutableMap.copyOf(inputSpec), null, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java index 7069e9a78de3..530c0f657a51 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/JobHelperTest.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.TestHelper; @@ -68,27 +67,30 @@ public class JobHelperTest { private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); - 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")) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - null - ) - ), - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - JSON_MAPPER - ); + + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + JSON_MAPPER.convertValue( + new HadoopyStringInputRowParser( + new JSONParseSpec( + new TimestampSpec("t", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + null + ) + ), + JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT + ) + ) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(JSON_MAPPER) + .build(); private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( JSON_MAPPER.convertValue( @@ -123,27 +125,34 @@ public void setup() throws Exception dataFile = temporaryFolder.newFile(); config = new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - "website", - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), - null, - ImmutableList.of("timestamp", "host", "visited_num"), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource("website") + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), + null, + ImmutableList.of("timestamp", "host", "visited_num"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(this.interval) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java index e7f1402606ff..452a658e9d48 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/hadoop/DatasourceRecordReaderSegmentReaderTest.java @@ -165,7 +165,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception Assert.assertEquals(18, count); // Check the index - Assert.assertEquals(9, index.size()); + Assert.assertEquals(9, index.numRows()); final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test")); final List dimensions = index.getDimensionNames(false); Assert.assertEquals(2, dimensions.size()); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java index e8caea0256e0..75a4fe45eee6 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java @@ -44,7 +44,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -308,33 +307,34 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig() { return new HadoopDruidIndexerConfig( new HadoopIngestionSpec( - new DataSchema( - ingestionSpec1.getDataSource(), - HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( - new StringInputRowParser( - new CSVParseSpec( - new TimestampSpec("timestamp", "yyyyMMddHH", null), - DimensionsSpec.EMPTY, - null, - ImmutableList.of("timestamp", "host", "visited"), - false, - 0 - ), - null - ), - Map.class - ), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("visited_sum", "visited") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.NONE, - ImmutableList.of(Intervals.of("2000/3000")) - ), - null, - HadoopDruidIndexerConfig.JSON_MAPPER - ), + DataSchema.builder() + .withDataSource(ingestionSpec1.getDataSource()) + .withParserMap( + HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( + new StringInputRowParser( + new CSVParseSpec( + new TimestampSpec("timestamp", "yyyyMMddHH", null), + DimensionsSpec.EMPTY, + null, + ImmutableList.of("timestamp", "host", "visited"), + false, + 0 + ), + null + ), + Map.class + ) + ) + .withAggregators(new LongSumAggregatorFactory("visited_sum", "visited")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.NONE, + ImmutableList.of(Intervals.of("2000/3000")) + ) + ) + .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER) + .build(), new HadoopIOConfig( ImmutableMap.of( "paths", diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java index 8af77ca0e4fd..92bd8595560c 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/GranularityPathSpecTest.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.hadoop.mapreduce.Job; @@ -152,18 +151,17 @@ public void testAddInputPath() throws Exception { UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"})); HadoopIngestionSpec spec = new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z")) - ), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), new HadoopIOConfig(null, null, null), DEFAULT_TUNING_CONFIG ); @@ -204,18 +202,17 @@ public void testIntervalTrimming() throws Exception { UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"})); HadoopIngestionSpec spec = new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - Granularities.ALL, - ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z")) - ), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.ALL, + ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), new HadoopIOConfig(null, null, null), DEFAULT_TUNING_CONFIG ); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java index 06a1416ad83f..fa1b2f59f48e 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/StaticPathSpecTest.java @@ -24,7 +24,6 @@ import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.lib.input.MultipleInputs; @@ -54,7 +53,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, null, jsonMapper); + DataSchema schema = DataSchema.builder().withDataSource("ds").withObjectMapper(jsonMapper).build(); HadoopIOConfig io = new HadoopIOConfig(null, null, null); pathSpec.addInputPaths(new HadoopDruidIndexerConfig(new HadoopIngestionSpec(schema, io, null)), job); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java index f98287124ed0..8b4795d75f51 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/sampler/InputSourceSampler.java @@ -67,14 +67,13 @@ public class InputSourceSampler { private static final String SAMPLER_DATA_SOURCE = "sampler"; - private static final DataSchema DEFAULT_DATA_SCHEMA = new DataSchema( - SAMPLER_DATA_SOURCE, - new TimestampSpec(null, null, null), - new DimensionsSpec(null), - null, - null, - null - ); + + private static final DataSchema DEFAULT_DATA_SCHEMA = + DataSchema.builder() + .withDataSource(SAMPLER_DATA_SOURCE) + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .build(); // We want to be able to sort the list of processed results back into the same order that we read them from the input // source so that the rows in the data loader are not always changing. To do this, we add a temporary column to the diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java index f6732f68a6c4..ff0aa674ef3f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TestIndexTask.java @@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.indexing.DataSchema; @@ -62,7 +61,7 @@ public TestIndexTask( id, taskResource, new IndexIngestionSpec( - new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper), + DataSchema.builder().withDataSource(dataSource).withObjectMapper(mapper).build(), new IndexTask.IndexIOConfig( new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java index 1b742971eb95..377f4ece0657 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java @@ -939,18 +939,19 @@ private void runIndexTask(@Nullable PartitionsSpec partitionsSpec, boolean appen null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATA_SOURCE, - new TimestampSpec("ts", "auto", null), - new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim"))), - new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.MINUTE, - ImmutableList.of(INTERVAL_TO_INDEX) - ), - null - ), + DataSchema.builder() + .withDataSource(DATA_SOURCE) + .withTimestamp(new TimestampSpec("ts", "auto", null)) + .withDimensions(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim"))) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.MINUTE, + ImmutableList.of(INTERVAL_TO_INDEX) + ) + ) + .build(), ioConfig, tuningConfig ), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index c1bf649980f6..1f862cc4c38f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1913,6 +1913,7 @@ public Metadata getMetadata() null, null, null, + null, null ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java index ff828f16789d..0a72b77e1db7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/HadoopIndexTaskTest.java @@ -27,7 +27,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.server.security.Action; @@ -50,15 +49,19 @@ public void testCorrectInputSourceResources() final HadoopIndexTask task = new HadoopIndexTask( null, new HadoopIngestionSpec( - new DataSchema( - "foo", null, new AggregatorFactory[0], new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - jsonMapper - ), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), + new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), + null ), null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java index ab953ba954ac..d84aa154fd26 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexIngestionSpecTest.java @@ -25,7 +25,6 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.junit.Rule; @@ -45,14 +44,11 @@ public void testParserAndInputFormat() "Cannot use parser and inputSource together. Try using inputFormat instead of parser." ); final IndexIngestionSpec spec = new IndexIngestionSpec( - new DataSchema( - "dataSource", - ImmutableMap.of("fake", "parser map"), - new AggregatorFactory[0], - new ArbitraryGranularitySpec(Granularities.NONE, null), - null, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withParserMap(ImmutableMap.of("fake", "parser map")) + .withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null)) + .build(), new IndexIOConfig( new NoopInputSource(), new NoopInputFormat(), @@ -69,14 +65,11 @@ public void testParserAndInputSource() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot use parser and inputSource together."); final IndexIngestionSpec spec = new IndexIngestionSpec( - new DataSchema( - "dataSource", - ImmutableMap.of("fake", "parser map"), - new AggregatorFactory[0], - new ArbitraryGranularitySpec(Granularities.NONE, null), - null, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withParserMap(ImmutableMap.of("fake", "parser map")) + .withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null)) + .build(), new IndexIOConfig( new NoopInputSource(), null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index d03ccf465e57..defa2107b9ac 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -166,6 +166,25 @@ public class IndexTaskTest extends IngestionTestBase 0 ); + private static final DataSchema DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test-json") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions( + new StringDimensionSchema("ts"), + new StringDimensionSchema("dim"), + new LongDimensionSchema("valDim") + ) + .withAggregators(new LongSumAggregatorFactory("valMet", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/P1D")) + ) + ) + .build(); + @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { @@ -225,24 +244,7 @@ public void testCorrectInputSourceResources() { IndexTask indexTask = createIndexTask( new IndexIngestionSpec( - new DataSchema( - "test-json", - DEFAULT_TIMESTAMP_SPEC, - new DimensionsSpec( - ImmutableList.of( - new StringDimensionSchema("ts"), - new StringDimensionSchema("dim"), - new LongDimensionSchema("valDim") - ) - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/P1D")) - ), - null - ), + DATA_SCHEMA, new IndexIOConfig( new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, @@ -275,24 +277,7 @@ public void testIngestNullOnlyColumns() throws Exception IndexTask indexTask = createIndexTask( new IndexIngestionSpec( - new DataSchema( - "test-json", - DEFAULT_TIMESTAMP_SPEC, - new DimensionsSpec( - ImmutableList.of( - new StringDimensionSchema("ts"), - new StringDimensionSchema("dim"), - new LongDimensionSchema("valDim") - ) - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/P1D")) - ), - null - ), + DATA_SCHEMA, new IndexIOConfig( new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, @@ -337,24 +322,7 @@ public void testIngestNullOnlyColumns_storeEmptyColumnsOff_shouldNotStoreEmptyCo IndexTask indexTask = createIndexTask( new IndexIngestionSpec( - new DataSchema( - "test-json", - DEFAULT_TIMESTAMP_SPEC, - new DimensionsSpec( - ImmutableList.of( - new StringDimensionSchema("ts"), - new StringDimensionSchema("dim"), - new LongDimensionSchema("valDim") - ) - ), - new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/P1D")) - ), - null - ), + DATA_SCHEMA, new IndexIOConfig( new LocalInputSource(tmpDir, "druid*"), DEFAULT_INPUT_FORMAT, @@ -2698,20 +2666,20 @@ private static IndexIngestionSpec createIngestionSpec( if (inputFormat != null) { Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec"); return new IndexIngestionSpec( - new DataSchema( - DATASOURCE, - Preconditions.checkNotNull(timestampSpec, "timestampSpec"), - Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec != null ? granularitySpec : new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/2015")) - ), - transformSpec - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(Preconditions.checkNotNull(timestampSpec, "timestampSpec")) + .withDimensions(Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec")) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + granularitySpec != null ? granularitySpec : new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/2015")) + ) + ) + .withTransform(transformSpec) + .build(), new IndexIOConfig( new LocalInputSource(baseDir, "druid*"), inputFormat, @@ -2723,22 +2691,21 @@ private static IndexIngestionSpec createIngestionSpec( } else { parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC; return new IndexIngestionSpec( - new DataSchema( - DATASOURCE, - parseSpec.getTimestampSpec(), - parseSpec.getDimensionsSpec(), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec != null ? granularitySpec : new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2014/2015")) - ), - transformSpec, - null, - objectMapper - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(parseSpec.getTimestampSpec()) + .withDimensions(parseSpec.getDimensionsSpec()) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + granularitySpec != null ? granularitySpec : new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2014/2015")) + ) + ) + .withTransform(transformSpec) + .withObjectMapper(objectMapper) + .build(), new IndexIOConfig( new LocalInputSource(baseDir, "druid*"), createInputFormatFromParseSpec(parseSpec), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index e6ea0e1329ae..99b0f8e7a759 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -38,7 +38,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; @@ -220,18 +219,19 @@ public void testIndexTaskSerde() throws Exception null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), TuningConfigBuilder.forIndexTask() .withMaxRowsInMemory(10) @@ -288,18 +288,19 @@ public void testIndexTaskwithResourceSerde() throws Exception null, new TaskResource("rofl", 2), new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), TuningConfigBuilder.forIndexTask() .withMaxRowsInMemory(10) @@ -412,15 +413,19 @@ public void testHadoopIndexTaskSerde() throws Exception final HadoopIndexTask task = new HadoopIndexTask( null, new HadoopIngestionSpec( - new DataSchema( - "foo", null, new AggregatorFactory[0], new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - jsonMapper - ), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), + new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), + null ), null, null, @@ -454,19 +459,18 @@ public void testHadoopIndexTaskWithContextSerde() throws Exception final HadoopIndexTask task = new HadoopIndexTask( null, new HadoopIngestionSpec( - new DataSchema( - "foo", - null, - null, - new AggregatorFactory[0], - new UniformGranularitySpec( - Granularities.DAY, - null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - null, - jsonMapper - ), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null + DataSchema.builder() + .withDataSource("foo") + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(jsonMapper) + .build(), + new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), + null ), null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java index 44b8284f407d..6caab0a0652e 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractMultiPhaseParallelIndexingTest.java @@ -221,14 +221,13 @@ protected ParallelIndexSupervisorTask createTask( dropExisting ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - DEFAULT_METRICS_SPEC, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); @@ -241,14 +240,13 @@ protected ParallelIndexSupervisorTask createTask( dropExisting ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - parseSpec.getTimestampSpec(), - parseSpec.getDimensionsSpec(), - DEFAULT_METRICS_SPEC, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(parseSpec.getTimestampSpec()) + .withDimensions(parseSpec.getDimensionsSpec()) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java index b8c59d042a31..a21dbb84616b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionTaskKillTest.java @@ -41,7 +41,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -250,14 +249,13 @@ private ParallelIndexSupervisorTask createTestTask( null ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); @@ -271,16 +269,13 @@ private ParallelIndexSupervisorTask createTestTask( ); //noinspection unchecked ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - parseSpec.getTimestampSpec(), - parseSpec.getDimensionsSpec(), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(parseSpec.getTimestampSpec()) + .withDimensions(parseSpec.getDimensionsSpec()) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java index 0d19cd86e03c..3adc154bb226 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/MultiPhaseParallelIndexingWithNullColumnTest.java @@ -127,18 +127,19 @@ public void testIngestNullColumn() throws JsonProcessingException null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - DIMENSIONS_SPEC.withDimensions(dimensionSchemas), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - INTERVAL_TO_INDEX - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas)) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + INTERVAL_TO_INDEX + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), JSON_FORMAT, @@ -177,18 +178,21 @@ public void testIngestNullColumn_useFieldDiscovery_includeAllDimensions_shouldSt null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - new DimensionsSpec.Builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build(), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - INTERVAL_TO_INDEX - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build() + ) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + INTERVAL_TO_INDEX + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), new JsonInputFormat( @@ -237,18 +241,21 @@ public void testIngestNullColumn_explicitPathSpec_useFieldDiscovery_includeAllDi null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - new DimensionsSpec.Builder().setIncludeAllDimensions(true).build(), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - null - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder().setIncludeAllDimensions(true).build() + ) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + null + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), new JsonInputFormat( @@ -303,20 +310,23 @@ public void testIngestNullColumn_storeEmptyColumnsOff_shouldNotStoreEmptyColumns null, null, new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - TIMESTAMP_SPEC, - DIMENSIONS_SPEC.withDimensions( - DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim")) - ), - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - INTERVAL_TO_INDEX - ), - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions( + DIMENSIONS_SPEC.withDimensions( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim")) + ) + ) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + INTERVAL_TO_INDEX + ) + ) + .build(), new ParallelIndexIOConfig( getInputSource(), JSON_FORMAT, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java index cbf711469734..2ea9385a0145 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java @@ -36,7 +36,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -147,20 +146,19 @@ private ParallelIndexSupervisorTask newTask( final int numTotalSubTasks = inputSource.estimateNumSplits(new NoopInputFormat(), null); // set up ingestion spec final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), ioConfig, TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(numTotalSubTasks).build() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java index c9858b80847b..4587ef6ce7e3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskResourceTest.java @@ -45,7 +45,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -400,20 +399,19 @@ private TestSupervisorTask newTask( ) { final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), ioConfig, TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(NUM_SUB_TASKS).build() ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java index e11fb2ef001c..fe5188fec6c5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskSerdeTest.java @@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -220,16 +219,19 @@ ParallelIndexIngestionSpecBuilder partitionsSpec(PartitionsSpec partitionsSpec) ParallelIndexIngestionSpec build() { - DataSchema dataSchema = new DataSchema( - "dataSource", - TIMESTAMP_SPEC, - DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.MINUTE, inputIntervals), - null - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(TIMESTAMP_SPEC) + .withDimensions(DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + inputIntervals + ) + ) + .build(); ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder .forParallelIndexTask() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java index 57dbafa173f9..b908d274e6c3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskTest.java @@ -263,14 +263,11 @@ public void testFailToConstructWhenBothAppendToExistingAndForceGuaranteedRollupA .withLogParseExceptions(false) .build(); final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "datasource", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - null, - null, - null - ), + DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .build(), ioConfig, tuningConfig ); @@ -325,25 +322,24 @@ public void testFailToConstructWhenBothInputSourceAndParserAreSet() expectedException.expect(IAE.class); expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser."); new ParallelIndexIngestionSpec( - new DataSchema( - "datasource", - mapper.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - null, - null, - null - ) - ), - Map.class - ), - null, - null, - null, - mapper - ), + DataSchema.builder() + .withDataSource("datasource") + .withParserMap( + mapper.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec(null, null, null), + DimensionsSpec.EMPTY, + null, + null, + null + ) + ), + Map.class + ) + ) + .withObjectMapper(mapper) + .build(), ioConfig, tuningConfig ); @@ -559,14 +555,11 @@ public void testCompactionTaskDoesntCleanup() throws Exception .build(); final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "datasource", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - null, - null, - null - ), + DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java index 2455ce692b95..f93ea8d0a8b8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTestingFactory.java @@ -31,11 +31,9 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashPartitionFunction; import org.joda.time.Interval; @@ -97,16 +95,13 @@ static DataSchema createDataSchema(List granularitySpecInputIntervals) DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION)) ); - return new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - new AggregatorFactory[]{}, - granularitySpec, - TransformSpec.NONE, - null, - NESTED_OBJECT_MAPPER - ); + return DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withGranularity(granularitySpec) + .withObjectMapper(NESTED_OBJECT_MAPPER) + .build(); } static ParallelIndexIngestionSpec createIngestionSpec( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java index 96494b8ac794..814e3f646424 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionTaskKillTest.java @@ -39,7 +39,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -329,14 +328,13 @@ protected ParallelIndexSupervisorTask newTask( null ); ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - DATASOURCE, - timestampSpec, - dimensionsSpec, - new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, - granularitySpec, - null - ), + DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity(granularitySpec) + .build(), ioConfig, tuningConfig ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java index ac8371acaa0d..b51224908644 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java @@ -43,7 +43,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.DataSegmentsWithSchemas; @@ -391,20 +390,19 @@ public void testRunInParallelIngestNullColumn() null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2017-12/P1M")) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas)) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2017-12/P1M")) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, @@ -444,20 +442,19 @@ public void testRunInParallelIngestNullColumn_storeEmptyColumnsOff_shouldNotStor null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas), - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(Intervals.of("2017-12/P1M")) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas)) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(Intervals.of("2017-12/P1M")) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), DEFAULT_INPUT_FORMAT, @@ -785,21 +782,24 @@ public void testIngestBothExplicitAndImplicitDims() throws IOException null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DimensionsSpec.builder() - .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) - .setIncludeAllDimensions(true) - .build(), - new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder() + .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) + .setIncludeAllDimensions(true) + .build() + ) + .withAggregators(new CountAggregatorFactory("cnt")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( @@ -868,21 +868,24 @@ public void testIngestBothExplicitAndImplicitDimsSchemaDiscovery() throws IOExce null, null, new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DimensionsSpec.builder() - .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) - .useSchemaDiscovery(true) - .build(), - new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, - new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions( + DimensionsSpec.builder() + .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) + .useSchemaDiscovery(true) + .build() + ) + .withAggregators(new CountAggregatorFactory("cnt")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, "*.json", true), new JsonInputFormat( @@ -948,20 +951,19 @@ private ParallelIndexSupervisorTask newTask( final ParallelIndexIngestionSpec ingestionSpec; if (useInputFormatApi) { ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(new LongSumAggregatorFactory("val", "val")) + .withGranularity( + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource), DEFAULT_INPUT_FORMAT, @@ -972,18 +974,19 @@ private ParallelIndexSupervisorTask newTask( ); } else { ingestionSpec = new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - DEFAULT_TIMESTAMP_SPEC, - DEFAULT_DIMENSIONS_SPEC, - DEFAULT_METRICS_SPEC, - new UniformGranularitySpec( - segmentGranularity, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ), - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(DEFAULT_TIMESTAMP_SPEC) + .withDimensions(DEFAULT_DIMENSIONS_SPEC) + .withAggregators(DEFAULT_METRICS_SPEC) + .withGranularity( + new UniformGranularitySpec( + segmentGranularity, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ) + ) + .build(), new ParallelIndexIOConfig( new LocalInputSource(inputDir, inputSourceFilter), createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java index 4acc3d3f5912..577dce1255d0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTaskSpecTest.java @@ -26,7 +26,6 @@ import org.apache.druid.data.input.impl.LocalInputSource; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Resource; @@ -48,14 +47,11 @@ public class SinglePhaseSubTaskSpecTest "groupId", "supervisorTaskId", new ParallelIndexIngestionSpec( - new DataSchema( - "dataSource", - new TimestampSpec(null, null, null), - new DimensionsSpec(null), - new AggregatorFactory[0], - null, - null - ), + DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .build(), new ParallelIndexIOConfig( new LocalInputSource(new File("baseDir"), "filter"), new JsonInputFormat(null, null, null, null, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java index aea98e9e1036..fcb92543914a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/TombstoneHelperTest.java @@ -67,8 +67,7 @@ public void noTombstonesWhenNoDataInInputIntervalAndNoExistingSegments() throws GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false, Collections.singletonList(interval) ); - DataSchema dataSchema = - new DataSchema("test", null, null, null, granularitySpec, null); + DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build(); // no segments will be pushed when all rows are thrown away, assume that: List pushedSegments = Collections.emptyList(); @@ -93,8 +92,7 @@ public void tombstonesCreatedWhenNoDataInInputIntervalAndExistingSegments() thro GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false, Collections.singletonList(interval) ); - DataSchema dataSchema = - new DataSchema("test", null, null, null, granularitySpec, null); + DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build(); // no segments will be pushed when all rows are thrown away, assume that: List pushedSegments = Collections.emptyList(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java index 44850ad0558f..35aeef9715d3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/InputRowSchemasTest.java @@ -122,17 +122,19 @@ public void testFromDataSchema() new DoubleDimensionSchema("d5") ) ); - DataSchema schema = new DataSchema( - "dataSourceName", - new TimestampSpec(null, null, null), - dimensionsSpec, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null - ); + DataSchema schema = + DataSchema.builder() + .withDataSource("dataSourceName") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(dimensionsSpec) + .withAggregators( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null) + ) + .build(); InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); @@ -154,14 +156,12 @@ public void testFromDataSchemaWithNoAggregator() new DoubleDimensionSchema("d5") ) ); - DataSchema schema = new DataSchema( - "dataSourceName", - new TimestampSpec(null, null, null), - dimensionsSpec, - new AggregatorFactory[]{}, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null - ); + DataSchema schema = DataSchema.builder() + .withDataSource("dataSourceName") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(dimensionsSpec) + .withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)) + .build(); InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index ddbed6be7c74..522134c4556f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -109,7 +109,6 @@ import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9Factory; @@ -672,18 +671,19 @@ public void testIndexTask() null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) @@ -735,18 +735,18 @@ public void testIndexTaskFailure() null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P1D")) - ), - null, - mapper - ), + DataSchema.builder() + .withDataSource("foo") + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P1D")) + ) + ) + .withObjectMapper(mapper) + .build(), new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) @@ -1165,18 +1165,19 @@ public void testResumeTasks() throws Exception null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) @@ -1253,18 +1254,19 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception null, null, new IndexIngestionSpec( - new DataSchema( - "foo", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, - new UniformGranularitySpec( - Granularities.DAY, - null, - ImmutableList.of(Intervals.of("2010-01-01/P2D")) - ), - null - ), + DataSchema.builder() + .withDataSource("foo") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new DoubleSumAggregatorFactory("met", "met")) + .withGranularity( + new UniformGranularitySpec( + Granularities.DAY, + null, + ImmutableList.of(Intervals.of("2010-01-01/P2D")) + ) + ) + .build(), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), TuningConfigBuilder.forIndexTask() .withMaxRowsPerSegment(10000) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java index ee90a3335a17..816b1298aac2 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java @@ -549,14 +549,15 @@ public void testGetActiveTaskRedactsPassword() throws JsonProcessingException new NoopTaskContextEnricher() ); - final DataSchema dataSchema = new DataSchema( - "DS", - new TimestampSpec(null, null, null), - new DimensionsSpec(null), - null, - new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null), - null - ); + final DataSchema dataSchema = + DataSchema.builder() + .withDataSource("DS") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .withGranularity( + new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null) + ) + .build(); final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( new HttpInputSource(Collections.singletonList(URI.create("http://host.org")), "user", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java index e788545507cd..1730d4b638fb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/CsvInputSourceSamplerTest.java @@ -47,14 +47,11 @@ public void testCSVColumnAllNull() { final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970")); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); - final DataSchema dataSchema = new DataSchema( - "sampler", - timestampSpec, - dimensionsSpec, - null, - null, - null - ); + final DataSchema dataSchema = DataSchema.builder() + .withDataSource("sampler") + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .build(); final List strCsvRows = ImmutableList.of( "FirstName,LastName,Number,Gender", diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java index c486c15f0f23..0220aacd8922 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerDiscoveryTest.java @@ -66,14 +66,11 @@ public void testDiscoveredTypesNonStrictBooleans() final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), - new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - null, - null, - null - ), + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .build(), null ); @@ -131,14 +128,11 @@ public void testDiscoveredTypesStrictBooleans() final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), - new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - null, - null, - null - ), + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .build(), null ); @@ -189,14 +183,12 @@ public void testDiscoveredTypesStrictBooleans() public void testTypesClassicDiscovery() { final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n')); - final DataSchema dataSchema = new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().build(), - null, - null, - null - ); + final DataSchema dataSchema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions(DimensionsSpec.builder().build()) + .build(); final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), @@ -248,23 +240,20 @@ public void testTypesClassicDiscovery() public void testTypesNoDiscoveryExplicitSchema() { final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n')); - final DataSchema dataSchema = new DataSchema( - "test", - new TimestampSpec("t", null, null), - DimensionsSpec.builder().setDimensions( - ImmutableList.of(new StringDimensionSchema("string"), - new LongDimensionSchema("long"), - new DoubleDimensionSchema("double"), - new StringDimensionSchema("bool"), - new AutoTypeColumnSchema("variant", null), - new AutoTypeColumnSchema("array", null), - new AutoTypeColumnSchema("nested", null) - ) - ).build(), - null, - null, - null - ); + final DataSchema dataSchema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec("t", null, null)) + .withDimensions( + new StringDimensionSchema("string"), + new LongDimensionSchema("long"), + new DoubleDimensionSchema("double"), + new StringDimensionSchema("bool"), + new AutoTypeColumnSchema("variant", null), + new AutoTypeColumnSchema("array", null), + new AutoTypeColumnSchema("nested", null) + ) + .build(); final SamplerResponse response = inputSourceSampler.sample( inputSource, new JsonInputFormat(null, null, null, null, null), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java index 087b12cef40e..80d88e0be17c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/sampler/InputSourceSamplerTest.java @@ -1497,24 +1497,24 @@ private DataSchema createDataSchema( ) throws IOException { if (useInputFormatApi) { - return new DataSchema( - "sampler", - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec - ); + return DataSchema.builder() + .withDataSource("sampler") + .withTimestamp(timestampSpec) + .withDimensions(dimensionsSpec) + .withAggregators(aggregators) + .withGranularity(granularitySpec) + .withTransform(transformSpec) + .build(); } else { final Map parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec)); - return new DataSchema( - "sampler", - parserMap, - aggregators, - granularitySpec, - transformSpec, - OBJECT_MAPPER - ); + return DataSchema.builder() + .withDataSource("sampler") + .withParserMap(parserMap) + .withAggregators(aggregators) + .withGranularity(granularitySpec) + .withTransform(transformSpec) + .withObjectMapper(OBJECT_MAPPER) + .build(); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java index 0f280059e0a5..7f44d44a00db 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunnerAuthTest.java @@ -33,10 +33,8 @@ import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.AllGranularity; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; -import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; @@ -107,16 +105,13 @@ public Authorizer getAuthorizer(String name) } }; - DataSchema dataSchema = new DataSchema( - "datasource", - new TimestampSpec(null, null, null), - new DimensionsSpec(Collections.emptyList()), - new AggregatorFactory[]{}, - new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()), - TransformSpec.NONE, - null, - null - ); + DataSchema dataSchema = + DataSchema.builder() + .withDataSource("datasource") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(new DimensionsSpec(Collections.emptyList())) + .withGranularity(new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList())) + .build(); SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class); SeekableStreamIndexTaskIOConfig ioConfig = new TestSeekableStreamIndexTaskIOConfig(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java index 258ebff7b504..7346fe9fb6e1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -95,7 +95,6 @@ import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -164,25 +163,23 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport protected static final ObjectMapper OBJECT_MAPPER; protected static final DataSchema OLD_DATA_SCHEMA; - protected static final DataSchema NEW_DATA_SCHEMA = new DataSchema( - "test_ds", - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null - ); + protected static final DataSchema NEW_DATA_SCHEMA = + DataSchema.builder() + .withDataSource("test_ds") + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .build(); protected static final InputFormat INPUT_FORMAT = new JsonInputFormat( new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), @@ -211,37 +208,38 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport static { OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json")); - OLD_DATA_SCHEMA = new DataSchema( - "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - false - ), - StandardCharsets.UTF_8.name() - ), - Map.class - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER - ); + OLD_DATA_SCHEMA = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + false + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(OBJECT_MAPPER) + .build(); } public SeekableStreamIndexTaskTestBase( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java index 87cd196c268f..6510e2cfbdc5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSamplerSpecTest.java @@ -48,7 +48,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.parsers.JSONPathSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; @@ -106,36 +105,37 @@ private static List> gene @Test(timeout = 10_000L) public void testSampleWithInputRowParser() throws Exception { - final DataSchema dataSchema = new DataSchema( - "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ) - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of(), - false - ) - ), - Map.class - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER - ); + DataSchema dataSchema = DataSchema.builder() + .withDataSource("test_ds") + .withParserMap( + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ) + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of(), + false + ) + ), + Map.class + ) + ) + .withAggregators( + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + ) + .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)) + .withObjectMapper(OBJECT_MAPPER) + .build(); final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig( STREAM, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java index 3e0e46d7a033..c085c3fc04cb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamSupervisorSpecTest.java @@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -59,7 +58,6 @@ import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.query.DruidMetrics; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -1260,18 +1258,19 @@ private static DataSchema getDataSchema() dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - DATASOURCE, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scaleOut) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java index daf85ac39c99..684a56cd1412 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorStateTest.java @@ -31,7 +31,6 @@ import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; @@ -83,7 +82,6 @@ import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.query.DruidMetrics; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.RowIngestionMetersFactory; @@ -2562,18 +2560,19 @@ private static DataSchema getDataSchema() dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim2")); - return new DataSchema( - DATASOURCE, - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec(dimensions), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec( - Granularities.HOUR, - Granularities.NONE, - ImmutableList.of() - ), - null - ); + return DataSchema.builder() + .withDataSource(DATASOURCE) + .withTimestamp(new TimestampSpec("timestamp", "iso", null)) + .withDimensions(dimensions) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity( + new UniformGranularitySpec( + Granularities.HOUR, + Granularities.NONE, + ImmutableList.of() + ) + ) + .build(); } private static SeekableStreamSupervisorIOConfig getIOConfig() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java index 88249509ef97..61396fc7ae61 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/TaskAnnouncementTest.java @@ -28,7 +28,6 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.junit.Assert; import org.junit.Test; @@ -51,7 +50,7 @@ public void testBackwardsCompatibleSerde() throws Exception "theid", new TaskResource("rofl", 2), new IndexTask.IndexIngestionSpec( - new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()), + DataSchema.builder().withDataSource("foo").withObjectMapper(new DefaultObjectMapper()).build(), ioConfig, null ), diff --git a/processing/src/main/java/org/apache/druid/data/input/impl/AggregateProjectionSpec.java b/processing/src/main/java/org/apache/druid/data/input/impl/AggregateProjectionSpec.java new file mode 100644 index 000000000000..5d96ba497fa6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/data/input/impl/AggregateProjectionSpec.java @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Lists; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.utils.CollectionUtils; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * API type to specify an aggregating projection on {@link org.apache.druid.segment.incremental.IncrementalIndexSchema} + * + * Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of + * projections and need to extract out a base interface from this class. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonTypeName(AggregateProjectionSpec.TYPE_NAME) +public class AggregateProjectionSpec +{ + public static final String TYPE_NAME = "aggregate"; + + private final String name; + private final List groupingColumns; + private final VirtualColumns virtualColumns; + private final AggregatorFactory[] aggregators; + private final List ordering; + @Nullable + private final String timeColumnName; + + @JsonCreator + public AggregateProjectionSpec( + @JsonProperty("name") String name, + @JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns, + @JsonProperty("groupingColumns") @Nullable List groupingColumns, + @JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators + ) + { + this.name = name; + if (CollectionUtils.isNullOrEmpty(groupingColumns)) { + throw InvalidInput.exception("groupingColumns must not be null or empty"); + } + this.groupingColumns = groupingColumns; + this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns; + // in the future this should be expanded to support user specified ordering, but for now we compute it based on + // the grouping columns, which is consistent with how rollup ordering works for incremental index base table + final ProjectionOrdering ordering = computeOrdering(this.virtualColumns, this.groupingColumns); + this.ordering = ordering.ordering; + this.timeColumnName = ordering.timeColumnName; + this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public List getGroupingColumns() + { + return groupingColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public AggregatorFactory[] getAggregators() + { + return aggregators; + } + + @JsonProperty + public List getOrdering() + { + return ordering; + } + + @JsonIgnore + public AggregateProjectionMetadata.Schema toMetadataSchema() + { + return new AggregateProjectionMetadata.Schema( + name, + timeColumnName, + virtualColumns, + groupingColumns.stream().map(DimensionSchema::getName).collect(Collectors.toList()), + aggregators, + ordering + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AggregateProjectionSpec that = (AggregateProjectionSpec) o; + return Objects.equals(name, that.name) + && Objects.equals(groupingColumns, that.groupingColumns) + && Objects.equals(virtualColumns, that.virtualColumns) + && Objects.deepEquals(aggregators, that.aggregators) + && Objects.equals(ordering, that.ordering); + } + + @Override + public int hashCode() + { + return Objects.hash(name, groupingColumns, virtualColumns, Arrays.hashCode(aggregators), ordering); + } + + @Override + public String toString() + { + return "AggregateProjectionSpec{" + + "name='" + name + '\'' + + ", groupingColumns=" + groupingColumns + + ", virtualColumns=" + virtualColumns + + ", aggregators=" + Arrays.toString(aggregators) + + ", ordering=" + ordering + + '}'; + } + + + private static ProjectionOrdering computeOrdering(VirtualColumns virtualColumns, List groupingColumns) + { + final List ordering = Lists.newArrayListWithCapacity(groupingColumns.size()); + + String timeColumnName = null; + Granularity granularity = null; + // try to find the __time column equivalent, which might be a time_floor expression to model granularity + // bucketing. The time column is decided as the finest granularity on __time detected. If the projection does + // not have a time-like column, the granularity will be handled as ALL for the projection and all projection + // rows will use a synthetic timestamp of the minimum timestamp of the incremental index + for (final DimensionSchema dimension : groupingColumns) { + ordering.add(OrderBy.ascending(dimension.getName())); + if (ColumnHolder.TIME_COLUMN_NAME.equals(dimension.getName())) { + timeColumnName = dimension.getName(); + granularity = Granularities.NONE; + } else { + final VirtualColumn vc = virtualColumns.getVirtualColumn(dimension.getName()); + final Granularity maybeGranularity = Granularities.fromVirtualColumn(vc); + if (granularity == null && maybeGranularity != null) { + granularity = maybeGranularity; + timeColumnName = dimension.getName(); + } else if (granularity != null && maybeGranularity != null && maybeGranularity.isFinerThan(granularity)) { + granularity = maybeGranularity; + timeColumnName = dimension.getName(); + } + } + } + return new ProjectionOrdering(ordering, timeColumnName); + } + + private static final class ProjectionOrdering + { + private final List ordering; + @Nullable + private final String timeColumnName; + + private ProjectionOrdering(List ordering, @Nullable String timeColumnName) + { + this.ordering = ordering; + this.timeColumnName = timeColumnName; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java index 05150ea830fc..08de0a364ec3 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/granularity/Granularities.java @@ -23,9 +23,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Query; import org.apache.druid.query.expression.TimestampFloorExprMacro; +import org.apache.druid.segment.AggregateProjectionMetadata; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; @@ -160,4 +162,32 @@ public static ExpressionVirtualColumn toVirtualColumn(Granularity granularity, S ExprMacroTable.granularity() ); } + + /** + * Converts a virtual column with a single input time column into a {@link Granularity} if it is a + * {@link TimestampFloorExprMacro.TimestampFloorExpr}. + *

+ * IMPORTANT - this method DOES NOT VERIFY that the virtual column has a single input that is a time column + * ({@link ColumnHolder#TIME_COLUMN_NAME} or equivalent projection time column as defined by + * {@link AggregateProjectionMetadata.Schema#getTimeColumnName()}). Callers must verify this externally before + * calling this method by examining {@link VirtualColumn#requiredColumns()}. + *

+ * This method also does not handle other time expressions, or if the virtual column is just an identifier for a + * time column + */ + @Nullable + public static Granularity fromVirtualColumn(VirtualColumn virtualColumn) + { + if (virtualColumn instanceof ExpressionVirtualColumn) { + final ExpressionVirtualColumn expressionVirtualColumn = (ExpressionVirtualColumn) virtualColumn; + final Expr expr = expressionVirtualColumn.getParsedExpression().get(); + if (expr instanceof TimestampFloorExprMacro.TimestampFloorExpr) { + final TimestampFloorExprMacro.TimestampFloorExpr gran = (TimestampFloorExprMacro.TimestampFloorExpr) expr; + if (gran.getArg().getBindingIfIdentifier() != null) { + return gran.getGranularity(); + } + } + } + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java index db7346517976..1273aebc0198 100644 --- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java @@ -190,6 +190,12 @@ public void segment(String segmentIdentifier) setDimension("segment", segmentIdentifier); } + @Override + public void projection(String projection) + { + setDimension("projection", projection); + } + @Override public void identity(String identity) { diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index ced9f0d4e2d9..f3fc1a80f83c 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -88,6 +88,12 @@ public class QueryContexts public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit"; public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold"; public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled"; + + // projection context keys + public static final String NO_PROJECTIONS = "noProjections"; + public static final String FORCE_PROJECTION = "forceProjections"; + public static final String USE_PROJECTION = "useProjection"; + // Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the // query's runtime public static final String QUERY_RESOURCE_ID = "queryResourceId"; diff --git a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java index 6dfad11fae40..812b1dd9134b 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java @@ -243,6 +243,11 @@ public interface QueryMetrics> void segment(String segmentIdentifier); + /** + * If a projection was used during segment processing, set its name as the projection dimension + */ + void projection(String projection); + /** * @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were * used to construct {@link org.apache.druid.segment.BitmapOffset} or diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 8567c4b6d6c6..3d34500b7109 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -26,6 +26,8 @@ import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ColumnTypeFactory; import org.apache.druid.segment.column.ValueType; @@ -365,6 +367,38 @@ public AggregatorFactory withName(String newName) throw new UOE("Cannot change output name for AggregatorFactory[%s].", this.getClass().getName()); } + /** + * Check to see if we can make a 'combining' factory of this aggregator that is suitable to process input from a + * selector of values produced by the other {@link AggregatorFactory} representing pre-aggregated data. Typically, + * this means that this and the other aggregator have the same inputs ({@link #requiredFields()}, and the same + * options for how the data was constructed into the intermediary type. If suitable, this method returns a + * 'combining' aggregator factory of this aggregator to use to process the pre-aggregated data which was produced by + * the other aggregator. + *

+ * This method is used indirectly in service of checking if a + * {@link org.apache.druid.segment.projections.QueryableProjection} can be used instead of the base table during + * {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)}, which checks if this + * aggregator can be substituted for its combining aggregator if and only if there exists a column that a cursor can + * read which was created by an aggregator that satisfies this method. In other words, this aggregator is the 'query' + * aggregator defined on the {@link CursorBuildSpec}, the argument to this method is the aggregator which created + * some column whose selectors are available to the cursor. If all aggregators on the {@link CursorBuildSpec} can be + * paired with aggregators from the underlying table in the cursor factory, then + * {@link CursorHolder#isPreAggregated()} will be set to true indicating that query engines should use this + * substituted aggregator instead of the original aggregators. + * + * @param preAggregated {@link AggregatorFactory} which produced the partially aggregated values which are + * available in a selector + * @return a "combining" {@link AggregatorFactory} to use with the pre-aggregated selector data + */ + @Nullable + public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated) + { + if (equals(preAggregated.withName(getName()))) { + return getCombiningFactory(); + } + return null; + } + /** * Merges the list of AggregatorFactory[] (presumable from metadata of some segments being merged) and * returns merged AggregatorFactory[] (for the metadata for merged segment). diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java index 4d12327c8965..c4c9a7875ef0 100755 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java @@ -454,13 +454,4 @@ public static boolean shouldUseObjectColumnAggregatorWrapper( } return false; } - - public static List getCombiningAggregators(List aggs) - { - List combining = new ArrayList<>(aggs.size()); - for (AggregatorFactory agg : aggs) { - combining.add(agg.getCombiningFactory()); - } - return combining; - } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java index 67583fc1fc0a..139bda6d50f3 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupingEngine.java @@ -20,6 +20,7 @@ package org.apache.druid.query.groupby; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; @@ -60,7 +61,6 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.ResultMergeQueryRunner; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.dimension.DefaultDimensionSpec; @@ -509,7 +509,7 @@ public Sequence process( final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec)); if (cursorHolder.isPreAggregated()) { - query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated())); } final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory); diff --git a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java index ef484df5f183..518611e408d1 100644 --- a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java @@ -151,6 +151,12 @@ public void segment(String segmentIdentifier) delegateQueryMetrics.segment(segmentIdentifier); } + @Override + public void projection(String projection) + { + delegateQueryMetrics.projection(projection); + } + @Override public void filterBundle(FilterBundle.BundleInfo bundleInfo) { diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java index dbec221248e9..4d4fe7f571ac 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryEngine.java @@ -20,6 +20,7 @@ package org.apache.druid.query.timeseries; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.inject.Inject; import org.apache.druid.collections.NonBlockingPool; @@ -38,7 +39,6 @@ import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorAdapters; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.vector.VectorCursorGranularizer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; @@ -104,7 +104,7 @@ public Sequence> process( final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics)); if (cursorHolder.isPreAggregated()) { - query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated())); } try { final Sequence> result; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java index d10d26242e3f..202a852c2be0 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryEngine.java @@ -19,6 +19,7 @@ package org.apache.druid.query.topn; +import com.google.common.base.Preconditions; import com.google.common.base.Predicates; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; @@ -30,7 +31,6 @@ import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor; import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory; @@ -89,7 +89,7 @@ public Sequence> query( final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics); final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec); if (cursorHolder.isPreAggregated()) { - query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs())); + query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated())); } final Cursor cursor = cursorHolder.asCursor(); if (cursor == null) { diff --git a/processing/src/main/java/org/apache/druid/segment/AggregateProjectionMetadata.java b/processing/src/main/java/org/apache/druid/segment/AggregateProjectionMetadata.java new file mode 100644 index 000000000000..7712cecb3034 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/AggregateProjectionMetadata.java @@ -0,0 +1,463 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; +import com.google.common.collect.Lists; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.projections.Projections; +import org.apache.druid.utils.CollectionUtils; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; + +/** + * Aggregate projection schema and row count information to store in {@link Metadata} which itself is stored inside a + * segment, defining which projections exist for the segment. + *

+ * Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of + * projections and need to extract out a base interface from this class. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonTypeName(AggregateProjectionSpec.TYPE_NAME) +public class AggregateProjectionMetadata +{ + private static final Interner SCHEMA_INTERNER = Interners.newWeakInterner(); + + public static final Comparator COMPARATOR = (o1, o2) -> { + int rowCompare = Integer.compare(o1.numRows, o2.numRows); + if (rowCompare != 0) { + return rowCompare; + } + return Schema.COMPARATOR.compare(o1.getSchema(), o2.getSchema()); + }; + + private final Schema schema; + private final int numRows; + + @JsonCreator + public AggregateProjectionMetadata( + @JsonProperty("schema") Schema schema, + @JsonProperty("numRows") int numRows + ) + { + this.schema = SCHEMA_INTERNER.intern(schema); + this.numRows = numRows; + } + + @JsonProperty + public Schema getSchema() + { + return schema; + } + + @JsonProperty + public int getNumRows() + { + return numRows; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + AggregateProjectionMetadata that = (AggregateProjectionMetadata) o; + return numRows == that.numRows && Objects.equals(schema, that.schema); + } + + @Override + public int hashCode() + { + return Objects.hash(schema, numRows); + } + + public static class Schema + { + /** + * It is not likely the best way to find the best matching projections, but it is the one we have for now. This + * comparator is used to sort all the projections in a segment "best" first, where best is defined as fewest grouping + * columns, most virtual columns and aggregators, as an approximation of likely to have the fewest number of rows to + * scan. + */ + public static final Comparator COMPARATOR = (o1, o2) -> { + // coarsest granularity first + if (o1.getGranularity().isFinerThan(o2.getGranularity())) { + return 1; + } + if (o2.getGranularity().isFinerThan(o1.getGranularity())) { + return -1; + } + // fewer dimensions first + final int dimsCompare = Integer.compare( + o1.groupingColumns.size(), + o2.groupingColumns.size() + ); + if (dimsCompare != 0) { + return dimsCompare; + } + // more metrics first + int metCompare = Integer.compare(o2.aggregators.length, o1.aggregators.length); + if (metCompare != 0) { + return metCompare; + } + // more virtual columns first + final int virtCompare = Integer.compare( + o2.virtualColumns.getVirtualColumns().length, + o1.virtualColumns.getVirtualColumns().length + ); + if (virtCompare != 0) { + return virtCompare; + } + return o1.name.compareTo(o2.name); + }; + + private final String name; + @Nullable + private final String timeColumnName; + private final VirtualColumns virtualColumns; + private final List groupingColumns; + private final AggregatorFactory[] aggregators; + private final List ordering; + private final List orderingWithTimeSubstitution; + + // computed fields + private final int timeColumnPosition; + private final Granularity granularity; + + @JsonCreator + public Schema( + @JsonProperty("name") String name, + @JsonProperty("timeColumnName") @Nullable String timeColumnName, + @JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns, + @JsonProperty("groupingColumns") List groupingColumns, + @JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators, + @JsonProperty("ordering") List ordering + ) + { + this.name = name; + if (CollectionUtils.isNullOrEmpty(groupingColumns)) { + throw InvalidInput.exception("groupingColumns must not be null or empty"); + } + this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns; + this.groupingColumns = groupingColumns; + this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators; + this.ordering = ordering; + + int foundTimePosition = -1; + this.orderingWithTimeSubstitution = Lists.newArrayListWithCapacity(ordering.size()); + Granularity granularity = null; + for (int i = 0; i < ordering.size(); i++) { + OrderBy orderBy = ordering.get(i); + if (orderBy.getColumnName().equals(timeColumnName)) { + orderingWithTimeSubstitution.add(new OrderBy(ColumnHolder.TIME_COLUMN_NAME, orderBy.getOrder())); + foundTimePosition = i; + timeColumnName = groupingColumns.get(foundTimePosition); + final VirtualColumn vc = this.virtualColumns.getVirtualColumn(groupingColumns.get(foundTimePosition)); + if (vc != null) { + granularity = Granularities.fromVirtualColumn(vc); + } else { + granularity = Granularities.NONE; + } + } else { + orderingWithTimeSubstitution.add(orderBy); + } + } + this.timeColumnName = timeColumnName; + this.timeColumnPosition = foundTimePosition; + this.granularity = granularity == null ? Granularities.ALL : granularity; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + @Nullable + public String getTimeColumnName() + { + return timeColumnName; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public VirtualColumns getVirtualColumns() + { + return virtualColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public List getGroupingColumns() + { + return groupingColumns; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public AggregatorFactory[] getAggregators() + { + return aggregators; + } + + @JsonProperty + public List getOrdering() + { + return ordering; + } + + @JsonIgnore + public List getOrderingWithTimeColumnSubstitution() + { + return orderingWithTimeSubstitution; + } + + @JsonIgnore + public int getTimeColumnPosition() + { + return timeColumnPosition; + } + + @JsonIgnore + public Granularity getGranularity() + { + return granularity; + } + + /** + * Check if this projection "matches" a {@link CursorBuildSpec} for a query to see if we can use a projection + * instead. For a projection to match, all grouping columns of the build spec must match, virtual columns of the + * build spec must either be available as a physical column on the projection, or the inputs to the virtual column + * must be available on the projection, and all aggregators must be compatible with pre-aggregated columns of the + * projection per {@link AggregatorFactory#substituteCombiningFactory(AggregatorFactory)}. If the projection + * matches, this method returns a {@link Projections.ProjectionMatch} which contains an updated + * {@link CursorBuildSpec} which has the remaining virtual columns from the original build spec which must still be + * computed and the 'combining' aggregator factories to process the pre-aggregated data from the projection, as well + * as a mapping of query column names to projection column names. + * + * @param queryCursorBuildSpec the {@link CursorBuildSpec} that contains the required inputs to build a + * {@link CursorHolder} for a query + * @param physicalColumnChecker Helper utility which can determine if a physical column required by + * queryCursorBuildSpec is available on the projection OR does not exist on the base + * table either + * @return a {@link Projections.ProjectionMatch} if the {@link CursorBuildSpec} matches the projection, which + * contains information such as which + */ + @Nullable + public Projections.ProjectionMatch matches( + CursorBuildSpec queryCursorBuildSpec, + Projections.PhysicalColumnChecker physicalColumnChecker + ) + { + if (!queryCursorBuildSpec.isCompatibleOrdering(orderingWithTimeSubstitution)) { + return null; + } + final List queryGrouping = queryCursorBuildSpec.getGroupingColumns(); + Projections.ProjectionMatchBuilder matchBuilder = new Projections.ProjectionMatchBuilder(); + + if (timeColumnName != null) { + matchBuilder.remapColumn(timeColumnName, ColumnHolder.TIME_COLUMN_NAME); + } + if (queryGrouping != null) { + for (String queryColumn : queryGrouping) { + matchBuilder = matchRequiredColumn( + matchBuilder, + queryColumn, + queryCursorBuildSpec.getVirtualColumns(), + physicalColumnChecker + ); + if (matchBuilder == null) { + return null; + } + } + } + if (queryCursorBuildSpec.getFilter() != null) { + for (String queryColumn : queryCursorBuildSpec.getFilter().getRequiredColumns()) { + matchBuilder = matchRequiredColumn( + matchBuilder, + queryColumn, + queryCursorBuildSpec.getVirtualColumns(), + physicalColumnChecker + ); + if (matchBuilder == null) { + return null; + } + } + } + if (!CollectionUtils.isNullOrEmpty(queryCursorBuildSpec.getAggregators())) { + boolean allMatch = true; + for (AggregatorFactory queryAgg : queryCursorBuildSpec.getAggregators()) { + boolean foundMatch = false; + for (AggregatorFactory projectionAgg : aggregators) { + final AggregatorFactory combining = queryAgg.substituteCombiningFactory(projectionAgg); + if (combining != null) { + matchBuilder.remapColumn(queryAgg.getName(), projectionAgg.getName()).addPreAggregatedAggregator(combining); + foundMatch = true; + } + } + allMatch = allMatch && foundMatch; + } + if (!allMatch) { + return null; + } + } + return matchBuilder.build(queryCursorBuildSpec); + } + + /** + * Ensure that the projection has the specified column required by a {@link CursorBuildSpec} in one form or another. + * If the column is a {@link VirtualColumn} on the build spec, ensure that the projection has an equivalent virtual + * column, or has the required inputs to compute the virtual column. If an equivalent virtual column exists, its + * name will be added to {@link Projections.ProjectionMatchBuilder#remapColumn(String, String)} so the query + * virtual column name can be mapped to the projection physical column name. If no equivalent virtual column exists, + * but the inputs are available on the projection to compute it, it will be added to + * {@link Projections.ProjectionMatchBuilder#addReferenceedVirtualColumn(VirtualColumn)}. + *

+ * Finally, if the column is not a virtual column in the query, it is checked with + * {@link Projections.PhysicalColumnChecker} which true if the column is present on the projection OR if the column + * is NOT present on the base table (meaning missing columns that do not exist anywhere do not disqualify a + * projection from being used). + * + * @param matchBuilder match state to add mappings of query virtual columns to projection physical columns + * and query virtual columns which still must be computed from projection physical + * columns + * @param column Column name to check + * @param queryVirtualColumns {@link VirtualColumns} from the {@link CursorBuildSpec} required by the query + * @param physicalColumnChecker Helper to check if the physical column exists on a projection, or does not exist on + * the base table + * @return {@link Projections.ProjectionMatchBuilder} with updated state per the rules described above, or null + * if the column cannot be matched + */ + @Nullable + private Projections.ProjectionMatchBuilder matchRequiredColumn( + Projections.ProjectionMatchBuilder matchBuilder, + String column, + VirtualColumns queryVirtualColumns, + Projections.PhysicalColumnChecker physicalColumnChecker + ) + { + final VirtualColumn buildSpecVirtualColumn = queryVirtualColumns.getVirtualColumn(column); + if (buildSpecVirtualColumn != null) { + // check to see if we have an equivalent virtual column defined in the projection, if so we can + final VirtualColumn projectionEquivalent = virtualColumns.findEquivalent(buildSpecVirtualColumn); + if (projectionEquivalent != null) { + if (!buildSpecVirtualColumn.getOutputName().equals(projectionEquivalent.getOutputName())) { + matchBuilder.remapColumn( + buildSpecVirtualColumn.getOutputName(), + projectionEquivalent.getOutputName() + ); + } + return matchBuilder; + } + + matchBuilder.addReferenceedVirtualColumn(buildSpecVirtualColumn); + final List requiredInputs = buildSpecVirtualColumn.requiredColumns(); + if (requiredInputs.size() == 1 && ColumnHolder.TIME_COLUMN_NAME.equals(requiredInputs.get(0))) { + // special handle time granularity. in the future this should be reworked to push this concept into the + // virtual column and underlying expression itself, but this will do for now + final Granularity virtualGranularity = Granularities.fromVirtualColumn(buildSpecVirtualColumn); + if (virtualGranularity != null) { + if (virtualGranularity.isFinerThan(granularity)) { + return null; + } + return matchBuilder.remapColumn(column, timeColumnName); + } else { + // anything else with __time requires none granularity + if (Granularities.NONE.equals(granularity)) { + return matchBuilder; + } + return null; + } + } else { + for (String required : requiredInputs) { + matchBuilder = matchRequiredColumn( + matchBuilder, + required, + queryVirtualColumns, + physicalColumnChecker + ); + if (matchBuilder == null) { + return null; + } + } + return matchBuilder; + } + } else { + if (physicalColumnChecker.check(name, column)) { + return matchBuilder; + } + return null; + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Schema schema = (Schema) o; + return Objects.equals(name, schema.name) + && Objects.equals(timeColumnName, schema.timeColumnName) + && Objects.equals(virtualColumns, schema.virtualColumns) + && Objects.equals(groupingColumns, schema.groupingColumns) + && Objects.deepEquals(aggregators, schema.aggregators) + && Objects.equals(ordering, schema.ordering); + } + + @Override + public int hashCode() + { + return Objects.hash( + name, + timeColumnName, + virtualColumns, + groupingColumns, + Arrays.hashCode(aggregators), + ordering + ); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index a2230cb2972c..d0209b0bfea8 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -27,6 +27,7 @@ import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Numbers; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.math.expr.Evals; @@ -360,8 +361,42 @@ public ColumnValueSelector makeColumnValueSelector( return rootLiteralSelector; } - return new ObjectColumnSelector() + return new ColumnValueSelector() { + @Override + public double getDouble() + { + Object o = StructuredData.unwrap(getObject()); + return Numbers.tryParseDouble(o, 0.0); + } + + @Override + public float getFloat() + { + Object o = StructuredData.unwrap(getObject()); + return Numbers.tryParseFloat(o, 0.0f); + } + + @Override + public long getLong() + { + Object o = StructuredData.unwrap(getObject()); + return Numbers.tryParseLong(o, 0L); + } + + @Override + public boolean isNull() + { + final Object o = StructuredData.unwrap(getObject()); + if (o instanceof Number) { + return false; + } + if (o instanceof String) { + return GuavaUtils.tryParseLong((String) o) == null && Doubles.tryParse((String) o) == null; + } + return true; + } + @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java index 7ce9709f30f6..c2bba50b3d88 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java @@ -75,6 +75,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 SimpleDictionaryMergingIterator.makePeekingComparator(); private final String name; + private final String outputName; private final IndexSpec indexSpec; private final SegmentWriteOutMedium segmentWriteOutMedium; private final Closer closer; @@ -84,9 +85,23 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 @Nullable private final ColumnType castToType; private boolean isVariantType = false; - + private byte variantTypeByte = 0x00; + + /** + * @param name column name + * @param outputName output smoosh file name. if this is a base table column, it will be the equivalent to + * name, however if this merger is for a projection, this will be prefixed with the + * projection name so that multiple projections can store the same column name at + * different smoosh file "paths" + * @param castToType optional mechanism to enforce that all values are a specific type + * @param indexSpec segment level storage options such as compression format and bitmap type + * @param segmentWriteOutMedium temporary storage location to stage segment outputs before finalizing into the segment + * @param closer resource closer if this merger needs to attach any closables that should be cleaned up + * when the segment is finished writing + */ public AutoTypeColumnMerger( String name, + String outputName, @Nullable ColumnType castToType, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, @@ -95,6 +110,7 @@ public AutoTypeColumnMerger( { this.name = name; + this.outputName = outputName; this.castToType = castToType; this.indexSpec = indexSpec; this.segmentWriteOutMedium = segmentWriteOutMedium; @@ -165,7 +181,7 @@ public void writeMergedValueDictionary(List adapters) throws I if (explicitType == null && !forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) { logicalType = ColumnType.STRING; serializer = new ScalarStringColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -179,7 +195,7 @@ public void writeMergedValueDictionary(List adapters) throws I switch (logicalType.getType()) { case LONG: serializer = new ScalarLongColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -187,7 +203,7 @@ public void writeMergedValueDictionary(List adapters) throws I break; case DOUBLE: serializer = new ScalarDoubleColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -195,7 +211,7 @@ public void writeMergedValueDictionary(List adapters) throws I break; case STRING: serializer = new ScalarStringColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -203,7 +219,7 @@ public void writeMergedValueDictionary(List adapters) throws I break; case ARRAY: serializer = new VariantColumnSerializer( - name, + outputName, logicalType, null, indexSpec, @@ -222,6 +238,7 @@ public void writeMergedValueDictionary(List adapters) throws I // mixed type column, but only root path, we can use VariantArrayColumnSerializer // pick the least restrictive type for the logical type isVariantType = true; + variantTypeByte = rootTypes.getByteValue(); for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) { logicalType = ColumnType.leastRestrictiveType(logicalType, type); } @@ -230,9 +247,9 @@ public void writeMergedValueDictionary(List adapters) throws I logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType); } serializer = new VariantColumnSerializer( - name, + outputName, null, - rootTypes.getByteValue(), + variantTypeByte, indexSpec, segmentWriteOutMedium, closer @@ -241,7 +258,7 @@ public void writeMergedValueDictionary(List adapters) throws I // all the bells and whistles logicalType = ColumnType.NESTED_DATA; serializer = new NestedDataColumnSerializer( - name, + outputName, indexSpec, segmentWriteOutMedium, closer @@ -262,7 +279,7 @@ public void writeMergedValueDictionary(List adapters) throws I sortedLookup.getSortedDoubles(), () -> new ArrayDictionaryMergingIterator( sortedArrayLookups, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); stringCardinality = sortedLookup.getStringCardinality(); @@ -284,7 +301,7 @@ public void writeMergedValueDictionary(List adapters) throws I ); final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator( sortedArrayLookups, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ); serializer.serializeDictionaries( () -> stringIterator, @@ -367,6 +384,76 @@ public ColumnDescriptor makeColumnDescriptor() return descriptorBuilder.build(); } + protected DictionaryIdLookup getIdLookup() + { + return serializer.getDictionaryIdLookup(); + } + + @Override + public void attachParent(DimensionMergerV9 parent, List projectionAdapters) throws IOException + { + DruidException.conditionalDefensive( + parent instanceof AutoTypeColumnMerger, + "Projection parent dimension must be same type, got [%s]", + parent.getClass() + ); + AutoTypeColumnMerger autoParent = (AutoTypeColumnMerger) parent; + logicalType = autoParent.logicalType; + isVariantType = autoParent.isVariantType; + if (autoParent.serializer instanceof ScalarStringColumnSerializer) { + serializer = new ScalarStringColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else if (autoParent.serializer instanceof ScalarLongColumnSerializer) { + serializer = new ScalarLongColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else if (autoParent.serializer instanceof ScalarDoubleColumnSerializer) { + serializer = new ScalarDoubleColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else if (autoParent.serializer instanceof VariantColumnSerializer) { + if (autoParent.isVariantType) { + serializer = new VariantColumnSerializer( + outputName, + null, + variantTypeByte, + indexSpec, + segmentWriteOutMedium, + closer + ); + } else { + serializer = new VariantColumnSerializer( + outputName, + logicalType, + null, + indexSpec, + segmentWriteOutMedium, + closer + ); + } + } else { + serializer = new NestedDataColumnSerializer( + outputName, + indexSpec, + segmentWriteOutMedium, + closer + ); + } + + serializer.setDictionaryIdLookup(autoParent.getIdLookup()); + serializer.open(); + } + public static class ArrayDictionaryMergingIterator implements Iterator { private static final Comparator> PEEKING_ITERATOR_COMPARATOR = diff --git a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java index a45c4b2d3e00..d4f6a47b05dd 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorBuildSpec.java @@ -26,6 +26,7 @@ import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.filter.Filter; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -54,10 +55,12 @@ public static CursorBuildSpecBuilder builder(CursorBuildSpec spec) private final VirtualColumns virtualColumns; @Nullable private final List aggregators; - private final List orderByColumns; + private final List preferredOrdering; private final QueryContext queryContext; + private final boolean isAggregate; + @Nullable private final QueryMetrics queryMetrics; @@ -77,9 +80,10 @@ public CursorBuildSpec( this.groupingColumns = groupingColumns; this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns"); this.aggregators = aggregators; - this.orderByColumns = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering"); + this.preferredOrdering = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering"); this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext"); this.queryMetrics = queryMetrics; + this.isAggregate = !CollectionUtils.isNullOrEmpty(groupingColumns) || !CollectionUtils.isNullOrEmpty(aggregators); } /** @@ -142,7 +146,7 @@ public List getAggregators() */ public List getPreferredOrdering() { - return orderByColumns; + return preferredOrdering; } /** @@ -166,6 +170,39 @@ public QueryMetrics getQueryMetrics() return queryMetrics; } + /** + * Returns true if {@link #getGroupingColumns()} is not null or empty and/or {@link #getAggregators()} is not null or + * empty. This method is useful for quickly checking if it is worth considering if a {@link CursorFactory} should + * attempt to produce a {@link CursorHolder} that is {@link CursorHolder#isPreAggregated()} to satisfy the build spec. + */ + public boolean isAggregate() + { + return isAggregate; + } + + /** + * Returns true if the supplied ordering matches {@link #getPreferredOrdering()}, meaning that the supplied ordering + * has everything which is in the preferred ordering in the same direction and order. The supplied ordering may have + * additional columns beyond the preferred ordering and still satisify this method. + */ + public boolean isCompatibleOrdering(List ordering) + { + // if the build spec doesn't prefer an ordering, any order is ok + if (preferredOrdering.isEmpty()) { + return true; + } + // all columns must be present in ordering if the build spec specifies them + if (ordering.size() < preferredOrdering.size()) { + return false; + } + for (int i = 0; i < preferredOrdering.size(); i++) { + if (!ordering.get(i).equals(preferredOrdering.get(i))) { + return false; + } + } + return true; + } + public static class CursorBuildSpecBuilder { @Nullable @@ -195,7 +232,7 @@ private CursorBuildSpecBuilder(CursorBuildSpec buildSpec) this.groupingColumns = buildSpec.groupingColumns; this.virtualColumns = buildSpec.virtualColumns; this.aggregators = buildSpec.aggregators; - this.preferredOrdering = buildSpec.orderByColumns; + this.preferredOrdering = buildSpec.preferredOrdering; this.queryContext = buildSpec.queryContext; this.queryMetrics = buildSpec.queryMetrics; } diff --git a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java index 79bf2b4e557a..808e1eb53846 100644 --- a/processing/src/main/java/org/apache/druid/segment/CursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/CursorHolder.java @@ -75,6 +75,18 @@ default boolean isPreAggregated() return false; } + /** + * Returns a set of replacement {@link AggregatorFactory} if and only if {@link #isPreAggregated()} is true. The + * query engine should replace the query aggregators with these aggregators, which are combining aggregators derived + * from the {@link CursorBuildSpec} passed into {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}. If + * {@link #isPreAggregated()} is not true, this method returns null + */ + @Nullable + default List getAggregatorsForPreAggregated() + { + return null; + } + /** * Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns * an empty list then the cursor has no defined ordering. diff --git a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java index b11498137fe7..a498e081b2cd 100644 --- a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java @@ -70,6 +70,7 @@ public abstract class DictionaryEncodedColumnMerger> imp private static final Logger log = new Logger(DictionaryEncodedColumnMerger.class); protected final String dimensionName; + protected final String outputName; protected final ProgressIndicator progress; protected final Closer closer; protected final IndexSpec indexSpec; @@ -81,6 +82,7 @@ public abstract class DictionaryEncodedColumnMerger> imp protected int rowCount = 0; protected int cardinality = 0; protected boolean hasNull = false; + protected boolean writeDictionary = true; @Nullable protected GenericIndexedWriter bitmapWriter; @@ -102,6 +104,7 @@ public abstract class DictionaryEncodedColumnMerger> imp public DictionaryEncodedColumnMerger( String dimensionName, + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -110,6 +113,7 @@ public DictionaryEncodedColumnMerger( ) { this.dimensionName = dimensionName; + this.outputName = outputName; this.indexSpec = indexSpec; this.capabilities = capabilities; this.segmentWriteOutMedium = segmentWriteOutMedium; @@ -171,8 +175,9 @@ public void writeMergedValueDictionary(List adapters) throws I numMergeIndex++; } - String dictFilename = StringUtils.format("%s.dim_values", dimensionName); + String dictFilename = StringUtils.format("%s.dim_values", outputName); dictionaryWriter = makeDictionaryWriter(dictFilename); + firstDictionaryValue = null; dictionarySize = 0; dictionaryWriter.open(); @@ -338,7 +343,7 @@ public void writeIndexes(@Nullable List segmentRowNumConversions) thr long dimStartTime = System.currentTimeMillis(); final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - String bmpFilename = StringUtils.format("%s.inverted", dimensionName); + String bmpFilename = StringUtils.format("%s.inverted", outputName); bitmapWriter = new GenericIndexedWriter<>( segmentWriteOutMedium, bmpFilename, @@ -402,11 +407,11 @@ protected void setupEncodedValueWriter() throws IOException { final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); - String filenameBase = StringUtils.format("%s.forward_dim", dimensionName); + String filenameBase = StringUtils.format("%s.forward_dim", outputName); if (capabilities.hasMultipleValues().isTrue()) { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create( - dimensionName, + outputName, segmentWriteOutMedium, filenameBase, cardinality, @@ -414,12 +419,12 @@ protected void setupEncodedValueWriter() throws IOException ); } else { encodedValueSerializer = - new VSizeColumnarMultiIntsSerializer(dimensionName, segmentWriteOutMedium, cardinality); + new VSizeColumnarMultiIntsSerializer(outputName, segmentWriteOutMedium, cardinality); } } else { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( - dimensionName, + outputName, segmentWriteOutMedium, filenameBase, cardinality, diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java index 853850625cbc..4ffc7ee04c4c 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java @@ -106,13 +106,17 @@ default MultiValueHandling getMultivalueHandling() * * See {@link DimensionMergerV9} interface for more information. * - * @param indexSpec Specification object for the index merge - * @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if needed - * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler - * @param progress ProgressIndicator used by the merging process + * @param outputName Output "file" name for the column to use for serializers, to control where it is + * stored in the segments internal files + * @param indexSpec Specification object for the index merge + * @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if + * needed + * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler + * @param progress ProgressIndicator used by the merging process * @return A new DimensionMergerV9 object. */ DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java index b5049a89817e..5e3f5cad8867 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionMergerV9.java @@ -21,6 +21,9 @@ import org.apache.druid.segment.column.ColumnDescriptor; +import java.io.IOException; +import java.util.List; + /** * Processing related interface * @@ -35,4 +38,13 @@ public interface DimensionMergerV9 extends DimensionMerger * @return ColumnDescriptor that IndexMergerV9 will use to build a column. */ ColumnDescriptor makeColumnDescriptor(); + + /** + * Attaches the {@link DimensionMergerV9} of a "projection" parent column so that stuff like value dictionaries can + * be shared between parent and child + */ + default void attachParent(DimensionMergerV9 parent, List projectionAdapters) throws IOException + { + // do nothing + } } diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java index e81667089227..035384e522b1 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java @@ -77,6 +77,7 @@ public DimensionIndexer makeIndexer(boolean useMaxMemory @Override public DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -85,7 +86,7 @@ public DimensionMergerV9 makeMerger( ) { return new DoubleDimensionMergerV9( - dimensionName, + outputName, indexSpec, segmentWriteOutMedium ); diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionMergerV9.java index ffa9530f1b73..c488947cc58b 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionMergerV9.java @@ -27,15 +27,15 @@ public class DoubleDimensionMergerV9 extends NumericDimensionMergerV9 { - DoubleDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) + DoubleDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) { - super(dimensionName, indexSpec, segmentWriteOutMedium); + super(outputName, indexSpec, segmentWriteOutMedium); } @Override GenericColumnSerializer setupEncodedValueWriter() { - return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); + return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java index 8d3471a4d924..925fefaa9be2 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java @@ -77,6 +77,7 @@ public DimensionIndexer makeIndexer(boolean useMaxMemoryEst @Override public DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -85,7 +86,7 @@ public DimensionMergerV9 makeMerger( ) { return new FloatDimensionMergerV9( - dimensionName, + outputName, indexSpec, segmentWriteOutMedium ); diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionMergerV9.java index eec1aa306582..f681091b9f73 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionMergerV9.java @@ -27,15 +27,15 @@ public class FloatDimensionMergerV9 extends NumericDimensionMergerV9 { - FloatDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) + FloatDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) { - super(dimensionName, indexSpec, segmentWriteOutMedium); + super(outputName, indexSpec, segmentWriteOutMedium); } @Override GenericColumnSerializer setupEncodedValueWriter() { - return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); + return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/IndexIO.java b/processing/src/main/java/org/apache/druid/segment/IndexIO.java index 4db49535ae60..9cab1ce19286 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexIO.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexIO.java @@ -48,6 +48,7 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnConfig; @@ -64,6 +65,7 @@ import org.apache.druid.segment.data.IndexedIterable; import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.data.VSizeColumnarMultiInts; +import org.apache.druid.segment.projections.Projections; import org.apache.druid.segment.serde.ComplexColumnPartSupplier; import org.apache.druid.segment.serde.FloatNumericColumnSupplier; import org.apache.druid.segment.serde.LongNumericColumnSupplier; @@ -621,6 +623,7 @@ public QueryableIndex load(File inDir, ObjectMapper mapper, boolean lazy, Segmen mapper, timeBuffer, smooshedFiles, + null, loadFailed ); @@ -643,6 +646,22 @@ public QueryableIndex load(File inDir, ObjectMapper mapper, boolean lazy, Segmen smooshedFiles, loadFailed ); + final Map>> projectionsColumns = new LinkedHashMap<>(); + final Metadata metadata = getMetdata(smooshedFiles, mapper, inDir); + if (metadata != null && metadata.getProjections() != null) { + for (AggregateProjectionMetadata projectionSpec : metadata.getProjections()) { + final Map> projectionColumns = readProjectionColumns( + mapper, + loadFailed, + projectionSpec, + smooshedFiles, + columns, + dataInterval + ); + + projectionsColumns.put(projectionSpec.getSchema().getName(), projectionColumns); + } + } final QueryableIndex index = new SimpleQueryableIndex( dataInterval, @@ -650,30 +669,15 @@ public QueryableIndex load(File inDir, ObjectMapper mapper, boolean lazy, Segmen segmentBitmapSerdeFactory.getBitmapFactory(), columns, smooshedFiles, - lazy + lazy, + metadata, + projectionsColumns ) { @Override public Metadata getMetadata() { - try { - ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); - if (metadataBB != null) { - return mapper.readValue( - SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), - Metadata.class - ); - } - } - catch (JsonParseException | JsonMappingException ex) { - // Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which - // is no longer supported then it is OK to not use the metadata instead of failing segment loading - log.warn(ex, "Failed to load metadata for segment [%s]", inDir); - } - catch (IOException ex) { - log.warn(ex, "Failed to read metadata for segment [%s]", inDir); - } - return null; + return getMetdata(smooshedFiles, mapper, inDir); } }; @@ -682,6 +686,87 @@ public Metadata getMetadata() return index; } + private Map> readProjectionColumns( + ObjectMapper mapper, + SegmentLazyLoadFailCallback loadFailed, + AggregateProjectionMetadata projectionSpec, + SmooshedFileMapper smooshedFiles, + Map> columns, + Interval dataInterval + ) throws IOException + { + final Map> projectionColumns = new LinkedHashMap<>(); + for (String groupingColumn : projectionSpec.getSchema().getGroupingColumns()) { + final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, groupingColumn); + final ByteBuffer colBuffer = smooshedFiles.mapFile(smooshName); + + final ColumnHolder parentColumn; + if (columns.containsKey(groupingColumn)) { + parentColumn = columns.get(groupingColumn).get(); + } else { + parentColumn = null; + } + registerColumnHolder( + true, + projectionColumns, + groupingColumn, + mapper, + colBuffer, + smooshedFiles, + parentColumn, + loadFailed + ); + + if (groupingColumn.equals(projectionSpec.getSchema().getTimeColumnName())) { + projectionColumns.put(ColumnHolder.TIME_COLUMN_NAME, projectionColumns.get(groupingColumn)); + } + } + for (AggregatorFactory aggregator : projectionSpec.getSchema().getAggregators()) { + final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, aggregator.getName()); + final ByteBuffer aggBuffer = smooshedFiles.mapFile(smooshName); + registerColumnHolder( + true, + projectionColumns, + aggregator.getName(), + mapper, + aggBuffer, + smooshedFiles, + null, + loadFailed + ); + } + if (projectionSpec.getSchema().getTimeColumnName() == null) { + projectionColumns.put( + ColumnHolder.TIME_COLUMN_NAME, + Projections.makeConstantTimeSupplier(projectionSpec.getNumRows(), dataInterval.getStartMillis()) + ); + } + return projectionColumns; + } + + @Nullable + private Metadata getMetdata(SmooshedFileMapper smooshedFiles, ObjectMapper mapper, File inDir) + { + try { + ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); + if (metadataBB != null) { + return mapper.readValue( + SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), + Metadata.class + ); + } + } + catch (JsonParseException | JsonMappingException ex) { + // Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which + // is no longer supported then it is OK to not use the metadata instead of failing segment loading + log.warn(ex, "Failed to load metadata for segment [%s]", inDir); + } + catch (IOException ex) { + log.warn(ex, "Failed to read metadata for segment [%s]", inDir); + } + return null; + } + /** * Return a list of columns that contains given inputs merged. The returned column names are in * the original order that is used when this segment is created. @@ -740,6 +825,7 @@ private void registerColumnHolders( mapper, colBuffer, smooshedFiles, + null, loadFailed ); } @@ -752,6 +838,7 @@ private void registerColumnHolder( ObjectMapper mapper, ByteBuffer colBuffer, SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parentColumn, SegmentLazyLoadFailCallback loadFailed ) throws IOException { @@ -767,7 +854,8 @@ private void registerColumnHolder( internedColumnName, mapper, colBuffer, - smooshedFiles + smooshedFiles, + parentColumn ); } catch (IOException | RuntimeException e) { @@ -782,7 +870,8 @@ private void registerColumnHolder( internedColumnName, mapper, colBuffer, - smooshedFiles + smooshedFiles, + parentColumn ); columns.put(internedColumnName, () -> columnHolder); } @@ -797,11 +886,12 @@ ColumnHolder deserializeColumn( String columnName, // columnName is not used in this method, but used in tests. ObjectMapper mapper, ByteBuffer byteBuffer, - SmooshedFileMapper smooshedFiles + SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parentColumn ) throws IOException { ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class); - return serde.read(byteBuffer, columnConfig, smooshedFiles); + return serde.read(byteBuffer, columnConfig, smooshedFiles, parentColumn); } } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java index f397b02814c4..1a7a587c8c1b 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMerger.java @@ -354,7 +354,7 @@ static TransformableRowIterator toMergedIndexRowIterator( // type of column doesn't have any kind of special per-index encoding that needs to be converted to the "global" // encoding. E. g. it's always true for subclasses of NumericDimensionMergerV9. //noinspection ObjectEquality - anySelectorChanged |= convertedDimensionSelector != sourceDimensionSelector; + anySelectorChanged = anySelectorChanged || convertedDimensionSelector != sourceDimensionSelector; convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues( indexNumber, diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java index fd3400d6c1fd..04cc579208d0 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -23,6 +23,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.google.inject.Inject; @@ -51,6 +52,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; import org.apache.druid.segment.loading.SegmentizerFactory; +import org.apache.druid.segment.projections.Projections; import org.apache.druid.segment.serde.ColumnPartSerde; import org.apache.druid.segment.serde.ComplexColumnPartSerde; import org.apache.druid.segment.serde.ComplexMetricSerde; @@ -64,6 +66,7 @@ import org.apache.druid.segment.serde.NullColumnPartSerde; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -138,7 +141,6 @@ private File makeIndexFiles( final DimensionsSpecInspector dimensionsSpecInspector, final List mergedMetrics, final Function, TimeAndDimsIterator> rowMergerFn, - final boolean fillRowNumConversions, final IndexSpec indexSpec, final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException @@ -154,7 +156,7 @@ private File makeIndexFiles( .filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim)) .collect(Collectors.toList()); - final Metadata segmentMetadata; + Metadata segmentMetadata; if (metricAggs != null) { AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length]; for (int i = 0; i < metricAggs.length; i++) { @@ -215,18 +217,20 @@ private File makeIndexFiles( mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats); final Map handlers = makeDimensionHandlers(mergedDimensions, dimFormats); + final Map mergersMap = Maps.newHashMapWithExpectedSize(mergedDimensions.size()); final List mergers = new ArrayList<>(); for (int i = 0; i < mergedDimensions.size(); i++) { DimensionHandler handler = handlers.get(mergedDimensions.get(i)); - mergers.add( - handler.makeMerger( - indexSpec, - segmentWriteOutMedium, - dimFormats.get(i).toColumnCapabilities(), - progress, - closer - ) + DimensionMergerV9 merger = handler.makeMerger( + mergedDimensions.get(i), + indexSpec, + segmentWriteOutMedium, + dimFormats.get(i).toColumnCapabilities(), + progress, + closer ); + mergers.add(merger); + mergersMap.put(mergedDimensions.get(i), merger); } /************* Setup Dim Conversions **************/ @@ -255,8 +259,7 @@ private File makeIndexFiles( timeAndDimsIterator, timeWriter, metricWriters, - mergers, - fillRowNumConversions + mergers ); /************ Create Inverted Indexes and Finalize Build Columns *************/ @@ -291,6 +294,20 @@ private File makeIndexFiles( progress.stopSection(section); + if (segmentMetadata != null && !CollectionUtils.isNullOrEmpty(segmentMetadata.getProjections())) { + segmentMetadata = makeProjections( + v9Smoosher, + segmentMetadata.getProjections(), + adapters, + indexSpec, + segmentWriteOutMedium, + progress, + closer, + mergersMap, + segmentMetadata + ); + } + /************* Make index.drd & metadata.drd files **************/ progress.progress(); makeIndexBinary( @@ -332,6 +349,193 @@ private void makeMetadataBinary( } } + private Metadata makeProjections( + final FileSmoosher smoosher, + final List projections, + final List adapters, + final IndexSpec indexSpec, + final SegmentWriteOutMedium segmentWriteOutMedium, + final ProgressIndicator progress, + final Closer closer, + final Map parentMergers, + final Metadata segmentMetadata + ) throws IOException + { + final List projectionMetadata = Lists.newArrayListWithCapacity(projections.size()); + for (AggregateProjectionMetadata spec : projections) { + final List projectionAdapters = Lists.newArrayListWithCapacity(adapters.size()); + final AggregateProjectionMetadata.Schema projectionSchema = spec.getSchema(); + for (IndexableAdapter adapter : adapters) { + projectionAdapters.add(adapter.getProjectionAdapter(projectionSchema.getName())); + } + // we can use the first adapter to get the dimensions and metrics because the projection schema should be + // identical across all segments. This is validated by segment metadata merging + final List dimensions = projectionAdapters.get(0).getDimensionNames(false); + final List metrics = Arrays.stream(projectionSchema.getAggregators()) + .map(AggregatorFactory::getName) + .collect(Collectors.toList()); + + + final List mergers = new ArrayList<>(); + final Map columnFormats = Maps.newLinkedHashMapWithExpectedSize(dimensions.size() + metrics.size()); + + for (String dimension : dimensions) { + final ColumnFormat dimensionFormat = projectionAdapters.get(0).getFormat(dimension); + columnFormats.put(dimension, dimensionFormat); + DimensionHandler handler = dimensionFormat.getColumnHandler(dimension); + DimensionMergerV9 merger = handler.makeMerger( + Projections.getProjectionSmooshV9FileName(spec, dimension), + indexSpec, + segmentWriteOutMedium, + dimensionFormat.toColumnCapabilities(), + progress, + closer + ); + if (parentMergers.containsKey(dimension)) { + merger.attachParent(parentMergers.get(dimension), projectionAdapters); + } else { + merger.writeMergedValueDictionary(projectionAdapters); + } + mergers.add(merger); + } + for (String metric : metrics) { + columnFormats.put(metric, projectionAdapters.get(0).getFormat(metric)); + } + + final GenericColumnSerializer timeWriter; + if (projectionSchema.getTimeColumnName() != null) { + timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); + } else { + timeWriter = null; + } + final ArrayList metricWriters = + setupMetricsWriters( + segmentWriteOutMedium, + metrics, + columnFormats, + indexSpec, + Projections.getProjectionSmooshV9Prefix(spec) + ); + + Function, TimeAndDimsIterator> rowMergerFn = + rowIterators -> new RowCombiningTimeAndDimsIterator(rowIterators, projectionSchema.getAggregators(), metrics); + + List perIndexRowIterators = Lists.newArrayListWithCapacity(projectionAdapters.size()); + for (int i = 0; i < projectionAdapters.size(); ++i) { + final IndexableAdapter adapter = projectionAdapters.get(i); + TransformableRowIterator target = adapter.getRows(); + perIndexRowIterators.add(IndexMerger.toMergedIndexRowIterator(target, i, mergers)); + } + final TimeAndDimsIterator timeAndDimsIterator = rowMergerFn.apply(perIndexRowIterators); + closer.register(timeAndDimsIterator); + + int rowCount = 0; + List rowNumConversions = new ArrayList<>(projectionAdapters.size()); + for (IndexableAdapter adapter : projectionAdapters) { + int[] arr = new int[adapter.getNumRows()]; + Arrays.fill(arr, INVALID_ROW); + rowNumConversions.add(IntBuffer.wrap(arr)); + } + + final String section = "walk through and merge projection[" + projectionSchema.getName() + "] rows"; + progress.startSection(section); + long startTime = System.currentTimeMillis(); + long time = startTime; + while (timeAndDimsIterator.moveToNext()) { + progress.progress(); + TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer(); + if (timeWriter != null) { + timeWriter.serialize(timeAndDims.timestampSelector); + } + + for (int metricIndex = 0; metricIndex < timeAndDims.getNumMetrics(); metricIndex++) { + metricWriters.get(metricIndex).serialize(timeAndDims.getMetricSelector(metricIndex)); + } + + for (int dimIndex = 0; dimIndex < timeAndDims.getNumDimensions(); dimIndex++) { + DimensionMergerV9 merger = mergers.get(dimIndex); + if (merger.hasOnlyNulls()) { + continue; + } + merger.processMergedRow(timeAndDims.getDimensionSelector(dimIndex)); + } + + RowCombiningTimeAndDimsIterator comprisedRows = (RowCombiningTimeAndDimsIterator) timeAndDimsIterator; + + for (int originalIteratorIndex = comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(0); + originalIteratorIndex >= 0; + originalIteratorIndex = + comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(originalIteratorIndex + 1)) { + + IntBuffer conversionBuffer = rowNumConversions.get(originalIteratorIndex); + int minRowNum = comprisedRows.getMinCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex); + int maxRowNum = comprisedRows.getMaxCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex); + + for (int rowNum = minRowNum; rowNum <= maxRowNum; rowNum++) { + while (conversionBuffer.position() < rowNum) { + conversionBuffer.put(INVALID_ROW); + } + conversionBuffer.put(rowCount); + } + } + if ((++rowCount % 500000) == 0) { + log.debug( + "walked 500,000/%d rows of projection[%s] in %,d millis.", + rowCount, + projectionSchema.getName(), + System.currentTimeMillis() - time + ); + time = System.currentTimeMillis(); + } + } + for (IntBuffer rowNumConversion : rowNumConversions) { + rowNumConversion.rewind(); + } + log.debug( + "completed walk through of %,d rows of projection[%s] in %,d millis.", + rowCount, + projectionSchema.getName(), + System.currentTimeMillis() - startTime + ); + progress.stopSection(section); + + final String section2 = "build projection[" + projectionSchema.getName() + "] inverted index and columns"; + progress.startSection(section2); + if (projectionSchema.getTimeColumnName() != null) { + makeTimeColumn( + smoosher, + progress, + timeWriter, + indexSpec, + Projections.getProjectionSmooshV9FileName(spec, projectionSchema.getTimeColumnName()) + ); + } + makeMetricsColumns( + smoosher, + progress, + metrics, + columnFormats, + metricWriters, + indexSpec, + Projections.getProjectionSmooshV9Prefix(spec) + ); + + for (int i = 0; i < dimensions.size(); i++) { + final String dimension = dimensions.get(i); + DimensionMergerV9 merger = mergers.get(i); + merger.writeIndexes(rowNumConversions); + if (!merger.hasOnlyNulls()) { + ColumnDescriptor columnDesc = merger.makeColumnDescriptor(); + makeColumn(smoosher, Projections.getProjectionSmooshV9FileName(spec, dimension), columnDesc); + } + } + + progress.stopSection(section2); + projectionMetadata.add(new AggregateProjectionMetadata(projectionSchema, rowCount)); + } + return segmentMetadata.withProjections(projectionMetadata); + } + private void makeIndexBinary( final FileSmoosher v9Smoosher, final List adapters, @@ -348,7 +552,7 @@ private void makeIndexBinary( columnSet.addAll(mergedMetrics); Preconditions.checkState( columnSet.size() == mergedDimensions.size() + mergedMetrics.size(), - "column names are not unique in dims%s and mets%s", + "column names are not unique in dims[%s] and mets[%s]", mergedDimensions, mergedMetrics ); @@ -444,6 +648,18 @@ private void makeMetricsColumns( final List metWriters, final IndexSpec indexSpec ) throws IOException + { + makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsTypes, metWriters, indexSpec, ""); + } + private void makeMetricsColumns( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final List mergedMetrics, + final Map metricsTypes, + final List metWriters, + final IndexSpec indexSpec, + final String namePrefix + ) throws IOException { final String section = "make metric columns"; progress.startSection(section); @@ -483,8 +699,9 @@ private void makeMetricsColumns( default: throw new ISE("Unknown type[%s]", type); } - makeColumn(v9Smoosher, metric, builder.build()); - log.debug("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime); + final String columnName = namePrefix + metric; + makeColumn(v9Smoosher, columnName, builder.build()); + log.debug("Completed metric column[%s] in %,d millis.", columnName, System.currentTimeMillis() - metricStartTime); } log.debug("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime); progress.stopSection(section); @@ -547,6 +764,17 @@ private void makeTimeColumn( final GenericColumnSerializer timeWriter, final IndexSpec indexSpec ) throws IOException + { + makeTimeColumn(v9Smoosher, progress, timeWriter, indexSpec, ColumnHolder.TIME_COLUMN_NAME); + } + + private void makeTimeColumn( + final FileSmoosher v9Smoosher, + final ProgressIndicator progress, + final GenericColumnSerializer timeWriter, + final IndexSpec indexSpec, + final String name + ) throws IOException { final String section = "make time column"; progress.startSection(section); @@ -557,7 +785,7 @@ private void makeTimeColumn( .setValueType(ValueType.LONG) .addSerde(createLongColumnPartSerde(timeWriter, indexSpec)) .build(); - makeColumn(v9Smoosher, ColumnHolder.TIME_COLUMN_NAME, serdeficator); + makeColumn(v9Smoosher, name, serdeficator); log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime); progress.stopSection(section); } @@ -601,23 +829,19 @@ private IndexMergeResult mergeIndexesAndWriteColumns( final TimeAndDimsIterator timeAndDimsIterator, final GenericColumnSerializer timeWriter, final ArrayList metricWriters, - final List mergers, - final boolean fillRowNumConversions + final List mergers ) throws IOException { final String section = "walk through and merge rows"; progress.startSection(section); long startTime = System.currentTimeMillis(); - List rowNumConversions = null; int rowCount = 0; - if (fillRowNumConversions) { - rowNumConversions = new ArrayList<>(adapters.size()); - for (IndexableAdapter adapter : adapters) { - int[] arr = new int[adapter.getNumRows()]; - Arrays.fill(arr, INVALID_ROW); - rowNumConversions.add(IntBuffer.wrap(arr)); - } + List rowNumConversions = new ArrayList<>(adapters.size()); + for (IndexableAdapter adapter : adapters) { + int[] arr = new int[adapter.getNumRows()]; + Arrays.fill(arr, INVALID_ROW); + rowNumConversions.add(IntBuffer.wrap(arr)); } long time = System.currentTimeMillis(); @@ -656,9 +880,7 @@ private IndexMergeResult mergeIndexesAndWriteColumns( } conversionBuffer.put(rowCount); } - } - } else if (timeAndDimsIterator instanceof MergingRowIterator) { RowPointer rowPointer = (RowPointer) timeAndDims; IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum()); @@ -668,11 +890,9 @@ private IndexMergeResult mergeIndexesAndWriteColumns( } conversionBuffer.put(rowCount); } else { - if (fillRowNumConversions) { - throw new IllegalStateException( - "Filling row num conversions is supported only with RowCombining and Merging iterators" - ); - } + throw new IllegalStateException( + "Filling row num conversions is supported only with RowCombining and Merging iterators" + ); } if ((++rowCount % 500000) == 0) { @@ -680,10 +900,8 @@ private IndexMergeResult mergeIndexesAndWriteColumns( time = System.currentTimeMillis(); } } - if (rowNumConversions != null) { - for (IntBuffer rowNumConversion : rowNumConversions) { - rowNumConversion.rewind(); - } + for (IntBuffer rowNumConversion : rowNumConversions) { + rowNumConversion.rewind(); } log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime); progress.stopSection(section); @@ -711,28 +929,40 @@ private ArrayList setupMetricsWriters( final Map metricsTypes, final IndexSpec indexSpec ) throws IOException + { + return setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricsTypes, indexSpec, ""); + } + + private ArrayList setupMetricsWriters( + final SegmentWriteOutMedium segmentWriteOutMedium, + final List mergedMetrics, + final Map metricsTypes, + final IndexSpec indexSpec, + final String prefix + ) throws IOException { ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); for (String metric : mergedMetrics) { TypeSignature type = metricsTypes.get(metric).getLogicalType(); + final String outputName = prefix + metric; GenericColumnSerializer writer; switch (type.getType()) { case LONG: - writer = createLongColumnSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); break; case FLOAT: - writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); break; case DOUBLE: - writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); break; case COMPLEX: ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(type.getComplexTypeName()); if (serde == null) { throw new ISE("Unknown type[%s]", type.getComplexTypeName()); } - writer = serde.getSerializer(segmentWriteOutMedium, metric, indexSpec); + writer = serde.getSerializer(segmentWriteOutMedium, outputName, indexSpec); break; default: throw new ISE("Unknown type[%s]", type); @@ -891,7 +1121,7 @@ public File persist( FileUtils.mkdirp(outDir); - log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size()); + log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.numRows()); return multiphaseMerge( Collections.singletonList( new IncrementalIndexAdapter( @@ -1179,7 +1409,6 @@ private File merge( new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec), mergedMetrics, rowMergerFn, - true, indexSpec, segmentWriteOutMediumFactory ); diff --git a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java index 023ca7001924..1d708e20844b 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexableAdapter.java @@ -64,6 +64,8 @@ public interface IndexableAdapter TransformableRowIterator getRows(); + IndexableAdapter getProjectionAdapter(String projection); + BitmapValues getBitmapValues(String dimension, int dictId); ColumnCapabilities getCapabilities(String column); diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java index 64a9f98cd44c..e93a71be8839 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java @@ -77,6 +77,7 @@ public DimensionIndexer makeIndexer(boolean useMaxMemoryEstima @Override public DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -85,7 +86,7 @@ public DimensionMergerV9 makeMerger( ) { return new LongDimensionMergerV9( - dimensionName, + outputName, indexSpec, segmentWriteOutMedium ); diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java index e4ab6ba3849c..b6a96e4f83c1 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionMergerV9.java @@ -27,15 +27,15 @@ public class LongDimensionMergerV9 extends NumericDimensionMergerV9 { - LongDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) + LongDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium) { - super(dimensionName, indexSpec, segmentWriteOutMedium); + super(outputName, indexSpec, segmentWriteOutMedium); } @Override GenericColumnSerializer setupEncodedValueWriter() { - return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec); + return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/Metadata.java b/processing/src/main/java/org/apache/druid/segment/Metadata.java index e21c673e2796..1da2ecce93c2 100644 --- a/processing/src/main/java/org/apache/druid/segment/Metadata.java +++ b/processing/src/main/java/org/apache/druid/segment/Metadata.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.granularity.Granularity; @@ -56,6 +57,8 @@ public class Metadata private final Boolean rollup; @Nullable private final List ordering; + @Nullable + private final List projections; public Metadata( @JsonProperty("container") @Nullable Map container, @@ -63,7 +66,8 @@ public Metadata( @JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec, @JsonProperty("queryGranularity") @Nullable Granularity queryGranularity, @JsonProperty("rollup") @Nullable Boolean rollup, - @JsonProperty("ordering") @Nullable List ordering + @JsonProperty("ordering") @Nullable List ordering, + @JsonProperty("projections") @Nullable List projections ) { this.container = container == null ? new ConcurrentHashMap<>() : container; @@ -72,6 +76,7 @@ public Metadata( this.queryGranularity = queryGranularity; this.rollup = rollup; this.ordering = ordering; + this.projections = projections; } @JsonProperty @@ -124,6 +129,27 @@ public List getOrdering() return ordering; } + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_EMPTY) + public List getProjections() + { + return projections; + } + + public Metadata withProjections(List projections) + { + return new Metadata( + container, + aggregators, + timestampSpec, + queryGranularity, + rollup, + ordering, + projections + ); + } + public Metadata putAll(@Nullable Map other) { if (other != null) { @@ -155,6 +181,7 @@ public static Metadata merge( List gransToMerge = new ArrayList<>(); List rollupToMerge = new ArrayList<>(); List> orderingsToMerge = new ArrayList<>(); + List> projectionsToMerge = new ArrayList<>(); for (Metadata metadata : toBeMerged) { if (metadata != null) { @@ -176,6 +203,7 @@ public static Metadata merge( } orderingsToMerge.add(metadata.getOrdering()); + projectionsToMerge.add(metadata.getProjections()); mergedContainer.putAll(metadata.container); } else { //if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then @@ -204,6 +232,7 @@ public static Metadata merge( Granularity.mergeGranularities(gransToMerge); final List mergedOrdering = mergeOrderings(orderingsToMerge); + validateProjections(projectionsToMerge); Boolean rollup = null; if (rollupToMerge != null && !rollupToMerge.isEmpty()) { @@ -227,7 +256,8 @@ public static Metadata merge( mergedTimestampSpec, mergedGranularity, rollup, - mergedOrdering + mergedOrdering, + projectionsToMerge.get(0) // we're going to replace this later with updated rowcount ); } @@ -246,13 +276,14 @@ public boolean equals(Object o) Objects.equals(timestampSpec, metadata.timestampSpec) && Objects.equals(queryGranularity, metadata.queryGranularity) && Objects.equals(rollup, metadata.rollup) && - Objects.equals(ordering, metadata.ordering); + Objects.equals(ordering, metadata.ordering) && + Objects.equals(projections, metadata.projections); } @Override public int hashCode() { - return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup); + return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup, ordering, projections); } @Override @@ -265,6 +296,7 @@ public String toString() ", queryGranularity=" + queryGranularity + ", rollup=" + rollup + ", ordering=" + ordering + + ", projections=" + projections + '}'; } @@ -308,4 +340,34 @@ public static List mergeOrderings(List> orderingsToMerge) mergedOrdering.add(orderBy); } } + + public static void validateProjections(List> projectionsToMerge) + { + final Map projectionsMap = new HashMap<>(); + // dedupe by name, fail if somehow incompatible projections are defined + int nullCount = 0; + int expectedSize = -1; + for (List projections : projectionsToMerge) { + if (projections == null) { + nullCount++; + continue; + } + if (expectedSize < 0) { + expectedSize = projections.size(); + } else if (projections.size() != expectedSize) { + throw DruidException.defensive("Unable to merge projections: mismatched projections count"); + } + for (AggregateProjectionMetadata projection : projections) { + AggregateProjectionMetadata prev = projectionsMap.putIfAbsent(projection.getSchema().getName(), projection); + if (prev != null && !prev.getSchema().equals(projection.getSchema())) { + throw DruidException.defensive("Unable to merge projections: mismatched projections [%s] and [%s]", prev, projection); + } + } + } + if (nullCount > 0) { + if (nullCount != projectionsToMerge.size()) { + throw DruidException.defensive("Unable to merge projections: some projections were null"); + } + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java index e679b2d2d9c7..348d40e4f2d2 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedCommonFormatColumnHandler.java @@ -77,6 +77,7 @@ public DimensionIndexer makeInde @Override public DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -84,7 +85,7 @@ public DimensionMergerV9 makeMerger( Closer closer ) { - return new AutoTypeColumnMerger(name, castTo, indexSpec, segmentWriteOutMedium, closer); + return new AutoTypeColumnMerger(name, outputName, castTo, indexSpec, segmentWriteOutMedium, closer); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnHandlerV4.java b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnHandlerV4.java index a63a18eac7d3..d74933faaf1e 100644 --- a/processing/src/main/java/org/apache/druid/segment/NestedDataColumnHandlerV4.java +++ b/processing/src/main/java/org/apache/druid/segment/NestedDataColumnHandlerV4.java @@ -73,6 +73,7 @@ public DimensionIndexer makeInde @Override public DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -80,7 +81,7 @@ public DimensionMergerV9 makeMerger( Closer closer ) { - return new NestedDataColumnMergerV4(name, indexSpec, segmentWriteOutMedium, closer); + return new NestedDataColumnMergerV4(outputName, indexSpec, segmentWriteOutMedium, closer); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/NumericDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/NumericDimensionMergerV9.java index 07e429b1d3de..e0a662e0a048 100644 --- a/processing/src/main/java/org/apache/druid/segment/NumericDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/NumericDimensionMergerV9.java @@ -31,19 +31,19 @@ */ public abstract class NumericDimensionMergerV9 implements DimensionMergerV9 { - protected final String dimensionName; + protected final String outputName; protected final IndexSpec indexSpec; protected final SegmentWriteOutMedium segmentWriteOutMedium; protected final GenericColumnSerializer serializer; NumericDimensionMergerV9( - String dimensionName, + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium ) { - this.dimensionName = dimensionName; + this.outputName = outputName; this.indexSpec = indexSpec; this.segmentWriteOutMedium = segmentWriteOutMedium; diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java index 309e36d24b87..ae019d735ab5 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndex.java @@ -24,6 +24,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.projections.QueryableProjection; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -85,4 +86,16 @@ default ColumnCapabilities getColumnCapabilities(String column) //@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed @Override void close(); + + @Nullable + default QueryableProjection getProjection(CursorBuildSpec cursorBuildSpec) + { + return null; + } + + @Nullable + default QueryableIndex getProjectionQueryableIndex(String name) + { + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java index 458923267117..8745761c183f 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorFactory.java @@ -20,13 +20,19 @@ package org.apache.druid.segment; import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.data.Offset; +import org.apache.druid.segment.projections.QueryableProjection; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; +import org.apache.druid.segment.vector.VectorOffset; import javax.annotation.Nullable; import java.util.LinkedHashSet; +import java.util.List; public class QueryableIndexCursorFactory implements CursorFactory { @@ -40,6 +46,46 @@ public QueryableIndexCursorFactory(QueryableIndex index) @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { + QueryableProjection projection = index.getProjection(spec); + if (projection != null) { + return new QueryableIndexCursorHolder(projection.getRowSelector(), projection.getCursorBuildSpec()) + { + @Override + protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset( + ColumnCache columnCache, + Offset baseOffset + ) + { + return projection.wrapColumnSelectorFactory( + super.makeColumnSelectorFactoryForOffset(columnCache, baseOffset) + ); + } + + @Override + protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( + ColumnCache columnCache, + VectorOffset baseOffset + ) + { + return projection.wrapVectorColumnSelectorFactory( + super.makeVectorColumnSelectorFactoryForOffset(columnCache, baseOffset) + ); + } + + @Override + public boolean isPreAggregated() + { + return true; + } + + @Nullable + @Override + public List getAggregatorsForPreAggregated() + { + return projection.getCursorBuildSpec().getAggregators(); + } + }; + } return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build()); } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java index 4b295931f0a8..54e482051ae4 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorHolder.java @@ -214,11 +214,9 @@ public Cursor asCursor() } final Offset baseCursorOffset = offset.clone(); - final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory( - virtualColumns, - Cursors.getTimeOrdering(ordering), - baseCursorOffset.getBaseReadableOffset(), - columnCache + final ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactoryForOffset( + columnCache, + baseCursorOffset ); // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter // needs to use a value matcher @@ -327,7 +325,20 @@ public void close() } - private VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( + protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset( + ColumnCache columnCache, + Offset baseOffset + ) + { + return new QueryableIndexColumnSelectorFactory( + virtualColumns, + Cursors.getTimeOrdering(ordering), + baseOffset.getBaseReadableOffset(), + columnCache + ); + } + + protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( ColumnCache columnCache, VectorOffset baseOffset ) diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index d9913036b5a6..fdec3477cd81 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -262,6 +262,14 @@ public RowIteratorImpl getRows() return new RowIteratorImpl(); } + @Override + public IndexableAdapter getProjectionAdapter(String projection) + { + QueryableIndex projectionIndex = input.getProjectionQueryableIndex(projection); + DruidException.conditionalDefensive(projectionIndex != null, "Projection[%s] was not found", projection); + return new QueryableIndexIndexableAdapter(projectionIndex); + } + /** * On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link * SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same diff --git a/processing/src/main/java/org/apache/druid/segment/RemapColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/RemapColumnSelectorFactory.java new file mode 100644 index 000000000000..4d4858a2aeab --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/RemapColumnSelectorFactory.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment; + +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.column.ColumnCapabilities; + +import javax.annotation.Nullable; +import java.util.Map; + +public class RemapColumnSelectorFactory implements ColumnSelectorFactory +{ + private final ColumnSelectorFactory delegate; + private final Map remap; + + public RemapColumnSelectorFactory(ColumnSelectorFactory delegate, Map remap) + { + this.delegate = delegate; + this.remap = remap; + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension())); + return delegate.makeDimensionSelector(remapDimensionSpec); + } + + @Override + public ColumnValueSelector makeColumnValueSelector(String columnName) + { + return delegate.makeColumnValueSelector(remap.getOrDefault(columnName, columnName)); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return delegate.getColumnCapabilities(remap.getOrDefault(column, column)); + } + + @Nullable + @Override + public RowIdSupplier getRowIdSupplier() + { + return delegate.getRowIdSupplier(); + } + + @Nullable + @Override + public ExpressionType getType(String name) + { + return delegate.getType(remap.getOrDefault(name, name)); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java index 9518876cb75c..6b48d304b80b 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java @@ -24,32 +24,45 @@ import com.google.common.base.Supplier; import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Interner; +import com.google.common.collect.Interners; import com.google.common.collect.Maps; +import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.query.OrderBy; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.data.ListIndexed; +import org.apache.druid.segment.projections.Projections; +import org.apache.druid.segment.projections.QueryableProjection; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.SortedSet; /** * */ public abstract class SimpleQueryableIndex implements QueryableIndex { + public static final Interner> ORDERING_INTERNER = Interners.newWeakInterner(); + private final Interval dataInterval; private final List columnNames; private final Indexed availableDimensions; private final BitmapFactory bitmapFactory; private final Map> columns; + private final List ordering; + private final Map projectionsMap; + private final SortedSet projections; + private final Map>> projectionColumns; private final SmooshedFileMapper fileMapper; private final Supplier> dimensionHandlers; - private final List ordering; public SimpleQueryableIndex( Interval dataInterval, @@ -59,6 +72,20 @@ public SimpleQueryableIndex( SmooshedFileMapper fileMapper, boolean lazy ) + { + this(dataInterval, dimNames, bitmapFactory, columns, fileMapper, lazy, null, null); + } + + public SimpleQueryableIndex( + Interval dataInterval, + Indexed dimNames, + BitmapFactory bitmapFactory, + Map> columns, + SmooshedFileMapper fileMapper, + boolean lazy, + @Nullable Metadata metadata, + @Nullable Map>> projectionColumns + ) { Preconditions.checkNotNull(columns.get(ColumnHolder.TIME_COLUMN_NAME)); this.dataInterval = Preconditions.checkNotNull(dataInterval, "dataInterval"); @@ -79,18 +106,35 @@ public SimpleQueryableIndex( this.columns = columns; this.fileMapper = fileMapper; + this.projectionColumns = projectionColumns == null ? Collections.emptyMap() : projectionColumns; + if (lazy) { this.dimensionHandlers = Suppliers.memoize(() -> initDimensionHandlers(availableDimensions)); } else { this.dimensionHandlers = () -> initDimensionHandlers(availableDimensions); } - - final Metadata metadata = getMetadata(); - if (metadata != null && metadata.getOrdering() != null) { - this.ordering = metadata.getOrdering(); + if (metadata != null) { + if (metadata.getOrdering() != null) { + this.ordering = ORDERING_INTERNER.intern(metadata.getOrdering()); + } else { + this.ordering = Cursors.ascendingTimeOrder(); + } + if (metadata.getProjections() != null) { + this.projectionsMap = Maps.newHashMapWithExpectedSize(metadata.getProjections().size()); + this.projections = new ObjectAVLTreeSet<>(AggregateProjectionMetadata.COMPARATOR); + for (AggregateProjectionMetadata projection : metadata.getProjections()) { + projections.add(projection); + projectionsMap.put(projection.getSchema().getName(), projection); + } + } else { + this.projectionsMap = Collections.emptyMap(); + this.projections = Collections.emptySortedSet(); + } } else { // When sort order isn't set in metadata.drd, assume the segment is sorted by __time. this.ordering = Cursors.ascendingTimeOrder(); + this.projections = Collections.emptySortedSet(); + this.projectionsMap = Collections.emptyMap(); } } @@ -177,4 +221,54 @@ private Map initDimensionHandlers(Indexed avai } return dimensionHandlerMap; } + + @Nullable + @Override + public QueryableProjection getProjection(CursorBuildSpec cursorBuildSpec) + { + return Projections.findMatchingProjection( + cursorBuildSpec, + projections, + (projectionName, columnName) -> + projectionColumns.get(projectionName).containsKey(columnName) || getColumnCapabilities(columnName) == null, + this::getProjectionQueryableIndex + ); + } + + @Override + public QueryableIndex getProjectionQueryableIndex(String name) + { + final AggregateProjectionMetadata projectionSpec = projectionsMap.get(name); + return new SimpleQueryableIndex( + dataInterval, + new ListIndexed<>(projectionSpec.getSchema().getGroupingColumns()), + bitmapFactory, + projectionColumns.get(name), + fileMapper, + true, + null, + null + ) + { + @Override + public Metadata getMetadata() + { + return null; + } + + @Override + public int getNumRows() + { + return projectionSpec.getNumRows(); + } + + @Override + public List getOrdering() + { + // return ordering with projection time column substituted with __time so query engines can treat it equivalently + return projectionSpec.getSchema().getOrderingWithTimeColumnSubstitution(); + } + }; + } } + diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java index 493660232319..3bb1e7217278 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java @@ -164,6 +164,7 @@ public DimensionIndexer makeIndexer(boolean useMaxMemory @Override public DimensionMergerV9 makeMerger( + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -182,6 +183,7 @@ public DimensionMergerV9 makeMerger( return new StringDimensionMergerV9( dimensionName, + outputName, indexSpec, segmentWriteOutMedium, capabilities, diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java index 109423af58f9..8c71d418556a 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java @@ -28,6 +28,7 @@ import org.apache.druid.collections.spatial.RTree; import org.apache.druid.collections.spatial.split.LinearGutmanSplitStrategy; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.column.ColumnCapabilities; @@ -63,8 +64,22 @@ public class StringDimensionMergerV9 extends DictionaryEncodedColumnMerger spatialWriter; + /** + * @param dimensionName column name + * @param outputName output smoosh file name. if this is a base table column, it will be the equivalent to + * name, however if this merger is for a projection, this will be prefixed with the + * projection name so that multiple projections can store the same column name at + * different smoosh file "paths" + * @param indexSpec segment level storage options such as compression format and bitmap type + * @param segmentWriteOutMedium temporary storage location to stage segment outputs before finalizing into the segment + * @param capabilities options for writing the column such as if we should write bitmap or spatial indexes + * @param progress hook to update status of what this merger is doing during segment persist and merging + * @param closer resource closer if this merger needs to attach any closables that should be cleaned up + * when the segment is finished writing + */ public StringDimensionMergerV9( String dimensionName, + String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, @@ -72,7 +87,7 @@ public StringDimensionMergerV9( Closer closer ) { - super(dimensionName, indexSpec, segmentWriteOutMedium, capabilities, progress, closer); + super(dimensionName, outputName, indexSpec, segmentWriteOutMedium, capabilities, progress, closer); } @Override @@ -113,7 +128,10 @@ protected DictionaryWriter makeDictionaryWriter(String fileName) @Override protected ExtendedIndexesMerger getExtendedIndexesMerger() { - return new SpatialIndexesMerger(); + if (capabilities.hasSpatialIndexes()) { + return new SpatialIndexesMerger(); + } + return null; } @Override @@ -127,9 +145,8 @@ public ColumnDescriptor makeColumnDescriptor() final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); builder.setValueType(ValueType.STRING); builder.setHasMultipleValues(hasMultiValue); - final DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde + DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde .serializerBuilder() - .withDictionary(dictionaryWriter) .withValue( encodedValueSerializer, hasMultiValue, @@ -140,11 +157,33 @@ public ColumnDescriptor makeColumnDescriptor() .withSpatialIndex(spatialWriter) .withByteOrder(IndexIO.BYTE_ORDER); + if (writeDictionary) { + partBuilder = partBuilder.withDictionary(dictionaryWriter); + } + return builder .addSerde(partBuilder.build()) .build(); } + @Override + public void attachParent(DimensionMergerV9 parent, List projectionAdapters) throws IOException + { + DruidException.conditionalDefensive( + parent instanceof StringDimensionMergerV9, + "Projection parent column must be same type, got [%s]", + parent.getClass() + ); + StringDimensionMergerV9 stringParent = (StringDimensionMergerV9) parent; + dictionarySize = stringParent.dictionarySize; + dimConversions = stringParent.dimConversions; + dictionaryWriter = stringParent.dictionaryWriter; + cardinality = dictionaryWriter.getCardinality(); + adapters = projectionAdapters; + setupEncodedValueWriter(); + writeDictionary = false; + } + /** * Write spatial indexes for string columns that have them */ diff --git a/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java b/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java index 4472ef099ae6..86fce26bb2ae 100644 --- a/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java +++ b/processing/src/main/java/org/apache/druid/segment/TimeAndDimsPointer.java @@ -94,8 +94,8 @@ public class TimeAndDimsPointer implements Comparable this.timestampSelector = timestampSelector; this.timePosition = timePosition; Preconditions.checkArgument( - timePosition >= 0 && timePosition <= dimensionSelectors.length, - "timePosition[%s] is out of range 0 .. [%s]", + timePosition <= dimensionSelectors.length, + "timePosition[%s] must be less than [%s]", timePosition, dimensionSelectors.length ); diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java index 276cbb03f1a5..08b268241bdf 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnDescriptor.java @@ -99,7 +99,8 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I public ColumnHolder read( ByteBuffer buffer, ColumnConfig columnConfig, - SmooshedFileMapper smooshedFiles + SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parent ) { final ColumnBuilder builder = new ColumnBuilder() @@ -108,9 +109,8 @@ public ColumnHolder read( .setFileMapper(smooshedFiles); for (ColumnPartSerde part : parts) { - part.getDeserializer().read(buffer, builder, columnConfig); + part.getDeserializer().read(buffer, builder, columnConfig, parent); } - return builder.build(); } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java index 11b472c26d30..94a41a9e862a 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnHolder.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.column; +import com.google.common.base.Supplier; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.selector.settable.SettableColumnValueSelector; @@ -45,8 +46,15 @@ default ColumnFormat getColumnFormat() } int getLength(); + BaseColumn getColumn(); + @Nullable + default Supplier getColumnSupplier() + { + return this::getColumn; + } + @Nullable ColumnIndexSupplier getIndexSupplier(); diff --git a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java index 30dc4a464ca2..ca38c34ed68b 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/SimpleColumnHolder.java @@ -99,6 +99,12 @@ public BaseColumn getColumn() return columnSupplier == null ? UnknownTypeComplexColumn.instance() : columnSupplier.get(); } + @Override + public Supplier getColumnSupplier() + { + return columnSupplier; + } + @Nullable @Override public ColumnIndexSupplier getIndexSupplier() diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/FactsHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/FactsHolder.java new file mode 100644 index 000000000000..f7eede101509 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/FactsHolder.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.incremental; + +import java.util.Comparator; +import java.util.Iterator; + +/** + * {@link IncrementalIndexRow} storage interface, a mutable data structure for building up a set or rows to eventually + * persist into an immutable segment + * + * @see IncrementalIndex for the data processor which constructs {@link IncrementalIndexRow} to store here + */ +public interface FactsHolder +{ + /** + * @return the previous rowIndex associated with the specified key, or + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. + */ + int getPriorIndex(IncrementalIndexRow key); + + /** + * Get minimum {@link IncrementalIndexRow#getTimestamp()} present in the facts holder + */ + long getMinTimeMillis(); + + /** + * Get maximum {@link IncrementalIndexRow#getTimestamp()} present in the facts holder + */ + long getMaxTimeMillis(); + + /** + * Get all {@link IncrementalIndex}, depending on the implementation, these rows may or may not be ordered in the same + * order they will be persisted in. Use {@link #persistIterable()} if this is required. + */ + Iterator iterator(boolean descending); + + /** + * Get all {@link IncrementalIndexRow} with {@link IncrementalIndexRow#getTimestamp()} between the start and end + * timestamps specified + */ + Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); + + /** + * Get all row {@link IncrementalIndexRow} 'keys', which is distinct groups if this is an aggregating facts holder or + * just every row present if not + */ + Iterable keySet(); + + /** + * Get all {@link IncrementalIndexRow} to persist, ordered with {@link Comparator } + */ + Iterable persistIterable(); + + /** + * @return the previous rowIndex associated with the specified key, or + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. + */ + int putIfAbsent(IncrementalIndexRow key, int rowIndex); + + /** + * Clear all rows present in the facts holder + */ + void clear(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index fc2a02c47b7b..9f8efaf73bc0 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java @@ -52,6 +52,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionIndexer; @@ -75,6 +76,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.projections.QueryableProjection; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; @@ -85,6 +87,7 @@ import javax.annotation.Nullable; import java.io.Closeable; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; @@ -105,7 +108,7 @@ * {@link IncrementalIndexCursorFactory} are thread-safe, and may be called concurrently with each other, and with * the "add" methods. This concurrency model supports real-time queries of the data in the index. */ -public abstract class IncrementalIndex implements Iterable, Closeable, ColumnInspector +public abstract class IncrementalIndex implements IncrementalIndexRowSelector, ColumnInspector, Iterable, Closeable { /** * Column selector used at ingestion time for inputs to aggregators. @@ -255,8 +258,9 @@ public ColumnCapabilities getColumnCapabilities(String columnName) private final boolean useSchemaDiscovery; - private final InputRowHolder inputRowHolder = new InputRowHolder(); + protected final InputRowHolder inputRowHolder = new InputRowHolder(); + @Nullable private volatile DateTime maxIngestedEventTime; /** @@ -362,11 +366,15 @@ protected IncrementalIndex( incrementalIndexSchema.getTimestampSpec(), this.queryGranularity, this.rollup, - getDimensionOrder().stream().map(OrderBy::ascending).collect(Collectors.toList()) + getDimensionOrder().stream().map(OrderBy::ascending).collect(Collectors.toList()), + Collections.emptyList() ); } - public abstract FactsHolder getFacts(); + @Nullable + public abstract QueryableProjection getProjection(CursorBuildSpec buildSpec); + + public abstract IncrementalIndexRowSelector getProjection(String name); public abstract boolean canAppendRow(); @@ -384,100 +392,11 @@ protected abstract AddToFactsResult addToFacts( boolean skipMaxRowsInMemoryCheck ) throws IndexSizeExceededException; - public abstract int getLastRowIndex(); - - protected abstract float getMetricFloatValue(int rowOffset, int aggOffset); - - protected abstract long getMetricLongValue(int rowOffset, int aggOffset); - - protected abstract Object getMetricObjectValue(int rowOffset, int aggOffset); - - protected abstract double getMetricDoubleValue(int rowOffset, int aggOffset); - - protected abstract boolean isNull(int rowOffset, int aggOffset); - - static class IncrementalIndexRowResult - { - private final IncrementalIndexRow incrementalIndexRow; - private final List parseExceptionMessages; - - IncrementalIndexRowResult(IncrementalIndexRow incrementalIndexRow, List parseExceptionMessages) - { - this.incrementalIndexRow = incrementalIndexRow; - this.parseExceptionMessages = parseExceptionMessages; - } - - IncrementalIndexRow getIncrementalIndexRow() - { - return incrementalIndexRow; - } - - List getParseExceptionMessages() - { - return parseExceptionMessages; - } - } - - static class AddToFactsResult - { - private final int rowCount; - private final long bytesInMemory; - private final List parseExceptionMessages; - - public AddToFactsResult( - int rowCount, - long bytesInMemory, - List parseExceptionMessages - ) - { - this.rowCount = rowCount; - this.bytesInMemory = bytesInMemory; - this.parseExceptionMessages = parseExceptionMessages; - } - - int getRowCount() - { - return rowCount; - } - - public long getBytesInMemory() - { - return bytesInMemory; - } - - public List getParseExceptionMessages() - { - return parseExceptionMessages; - } - } - - public static class InputRowHolder - { - @Nullable - private InputRow row; - private long rowId = -1; - - public void set(final InputRow row) - { - this.row = row; - this.rowId++; - } - - public void unset() - { - this.row = null; - } - - public InputRow getRow() - { - return Preconditions.checkNotNull(row, "row"); - } - public long getRowId() - { - return rowId; - } - } + public abstract Iterable iterableWithPostAggregations( + @Nullable List postAggs, + boolean descending + ); public boolean isRollup() { @@ -526,6 +445,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) } @Nullable + @Override public ColumnFormat getColumnFormat(String columnName) { if (timeAndMetricsColumnFormats.containsKey(columnName)) { @@ -746,23 +666,6 @@ public static ParseException getCombinedParseException( ); } - private static String getSimplifiedEventStringFromRow(InputRow inputRow) - { - if (inputRow instanceof MapBasedInputRow) { - return ((MapBasedInputRow) inputRow).getEvent().toString(); - } - - if (inputRow instanceof ListBasedInputRow) { - return ((ListBasedInputRow) inputRow).asMap().toString(); - } - - if (inputRow instanceof TransformedInputRow) { - InputRow innerRow = ((TransformedInputRow) inputRow).getBaseRow(); - return getSimplifiedEventStringFromRow(innerRow); - } - - return inputRow.toString(); - } private synchronized void updateMaxIngestedTime(DateTime eventTime) { @@ -771,12 +674,14 @@ private synchronized void updateMaxIngestedTime(DateTime eventTime) } } + @Override public boolean isEmpty() { return numEntries.get() == 0; } - public int size() + @Override + public int numRows() { return numEntries.get(); } @@ -824,6 +729,7 @@ public DimensionsSpec getDimensionsSpec() * * @param includeTime whether to include {@link ColumnHolder#TIME_COLUMN_NAME}. */ + @Override public List getDimensionNames(final boolean includeTime) { synchronized (dimensionDescs) { @@ -851,6 +757,7 @@ public List getDimensionNames(final boolean includeTime) /** * Returns a descriptor for each dimension. Does not inclue {@link ColumnHolder#TIME_COLUMN_NAME}. */ + @Override public List getDimensions() { synchronized (dimensionDescs) { @@ -861,6 +768,7 @@ public List getDimensions() /** * Returns the descriptor for a particular dimension. */ + @Override @Nullable public DimensionDesc getDimension(String dimension) { @@ -869,22 +777,45 @@ public DimensionDesc getDimension(String dimension) } } - public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) + @Override + @Nullable + public MetricDesc getMetric(String metric) + { + return metricDescs.get(metric); + } + + @Override + public List getOrdering() + { + return metadata.getOrdering(); + } + + @Override + public int getTimePosition() { - MetricDesc metricDesc = metricDescs.get(metric); + return timePosition; + } + + public static ColumnValueSelector makeMetricColumnValueSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + String metric + ) + { + final MetricDesc metricDesc = rowSelector.getMetric(metric); if (metricDesc == null) { return NilColumnValueSelector.instance(); } int metricIndex = metricDesc.getIndex(); switch (metricDesc.getCapabilities().getType()) { case COMPLEX: - return new ObjectMetricColumnSelector(metricDesc, currEntry, metricIndex); + return new ObjectMetricColumnSelector(rowSelector, currEntry, metricDesc); case LONG: - return new LongMetricColumnSelector(currEntry, metricIndex); + return new LongMetricColumnSelector(rowSelector, currEntry, metricIndex); case FLOAT: - return new FloatMetricColumnSelector(currEntry, metricIndex); + return new FloatMetricColumnSelector(rowSelector, currEntry, metricIndex); case DOUBLE: - return new DoubleMetricColumnSelector(currEntry, metricIndex); + return new DoubleMetricColumnSelector(rowSelector, currEntry, metricIndex); case STRING: throw new IllegalStateException("String is not a metric column type"); default: @@ -910,13 +841,6 @@ public DateTime getMaxTime() return isEmpty() ? null : DateTimes.utc(getMaxTimeMillis()); } - @Nullable - public Integer getDimensionIndex(String dimension) - { - DimensionDesc dimSpec = getDimension(dimension); - return dimSpec == null ? null : dimSpec.getIndex(); - } - /** * Returns names of time and dimension columns, in persist sort order. Includes {@link ColumnHolder#TIME_COLUMN_NAME}. */ @@ -956,7 +880,7 @@ public void loadDimensionIterable( ) { synchronized (dimensionDescs) { - if (size() != 0) { + if (numRows() != 0) { throw new ISE("Cannot load dimension order[%s] when existing index is not empty.", dimensionDescs.keySet()); } for (String dim : oldDimensionOrder) { @@ -983,6 +907,7 @@ private DimensionDesc initDimension(int dimensionIndex, String dimensionName, Di return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler, useMaxMemoryEstimates); } + @Override public List getMetricNames() { return ImmutableList.copyOf(metricDescs.keySet()); @@ -1003,6 +928,48 @@ public Metadata getMetadata() return metadata; } + @Override + public Iterator iterator() + { + return iterableWithPostAggregations(null, false).iterator(); + } + + public DateTime getMaxIngestedEventTime() + { + return maxIngestedEventTime; + } + + protected ColumnSelectorFactory makeColumnSelectorFactory( + @Nullable final AggregatorFactory agg, + final InputRowHolder in + ) + { + return makeColumnSelectorFactory(virtualColumns, in, agg); + } + + protected final Comparator dimsComparator() + { + return new IncrementalIndexRowComparator(timePosition, dimensionDescsList); + } + + private static String getSimplifiedEventStringFromRow(InputRow inputRow) + { + if (inputRow instanceof MapBasedInputRow) { + return ((MapBasedInputRow) inputRow).getEvent().toString(); + } + + if (inputRow instanceof ListBasedInputRow) { + return ((ListBasedInputRow) inputRow).asMap().toString(); + } + + if (inputRow instanceof TransformedInputRow) { + InputRow innerRow = ((TransformedInputRow) inputRow).getBaseRow(); + return getSimplifiedEventStringFromRow(innerRow); + } + + return inputRow.toString(); + } + private static AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] aggregators) { AggregatorFactory[] combiningAggregators = new AggregatorFactory[aggregators.length]; @@ -1012,30 +979,24 @@ private static AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] a return combiningAggregators; } - @Override - public Iterator iterator() - { - return iterableWithPostAggregations(null, false).iterator(); - } - - public abstract Iterable iterableWithPostAggregations( - @Nullable List postAggs, - boolean descending - ); - - public DateTime getMaxIngestedEventTime() + private static boolean allNull(Object[] dims, int startPosition) { - return maxIngestedEventTime; + for (int i = startPosition; i < dims.length; i++) { + if (dims[i] != null) { + return false; + } + } + return true; } public static final class DimensionDesc { private final int index; private final String name; - private final DimensionHandler handler; - private final DimensionIndexer indexer; + private final DimensionHandler handler; + private final DimensionIndexer indexer; - public DimensionDesc(int index, String name, DimensionHandler handler, boolean useMaxMemoryEstimates) + public DimensionDesc(int index, String name, DimensionHandler handler, boolean useMaxMemoryEstimates) { this.index = index; this.name = name; @@ -1043,6 +1004,14 @@ public DimensionDesc(int index, String name, DimensionHandler handler, boolean u this.indexer = handler.makeIndexer(useMaxMemoryEstimates); } + public DimensionDesc(int index, String name, DimensionHandler handler, DimensionIndexer indexer) + { + this.index = index; + this.name = name; + this.handler = handler; + this.indexer = indexer; + } + public int getIndex() { return index; @@ -1058,12 +1027,12 @@ public ColumnCapabilities getCapabilities() return indexer.getColumnCapabilities(); } - public DimensionHandler getHandler() + public DimensionHandler getHandler() { return handler; } - public DimensionIndexer getIndexer() + public DimensionIndexer getIndexer() { return indexer; } @@ -1124,19 +1093,90 @@ public ColumnCapabilities getCapabilities() } } - protected ColumnSelectorFactory makeColumnSelectorFactory( - @Nullable final AggregatorFactory agg, - final InputRowHolder in - ) + public static class AddToFactsResult { - return makeColumnSelectorFactory(virtualColumns, in, agg); + private final int rowCount; + private final long bytesInMemory; + private final List parseExceptionMessages; + + public AddToFactsResult( + int rowCount, + long bytesInMemory, + List parseExceptionMessages + ) + { + this.rowCount = rowCount; + this.bytesInMemory = bytesInMemory; + this.parseExceptionMessages = parseExceptionMessages; + } + + int getRowCount() + { + return rowCount; + } + + public long getBytesInMemory() + { + return bytesInMemory; + } + + public List getParseExceptionMessages() + { + return parseExceptionMessages; + } } - protected final Comparator dimsComparator() + public static class InputRowHolder { - return new IncrementalIndexRowComparator(timePosition, dimensionDescsList); + @Nullable + private InputRow row; + private long rowId = -1; + + public void set(final InputRow row) + { + this.row = row; + this.rowId++; + } + + public void unset() + { + this.row = null; + } + + public InputRow getRow() + { + return Preconditions.checkNotNull(row, "row"); + } + + public long getRowId() + { + return rowId; + } + } + + static class IncrementalIndexRowResult + { + private final IncrementalIndexRow incrementalIndexRow; + private final List parseExceptionMessages; + + IncrementalIndexRowResult(IncrementalIndexRow incrementalIndexRow, List parseExceptionMessages) + { + this.incrementalIndexRow = incrementalIndexRow; + this.parseExceptionMessages = parseExceptionMessages; + } + + IncrementalIndexRow getIncrementalIndexRow() + { + return incrementalIndexRow; + } + + List getParseExceptionMessages() + { + return parseExceptionMessages; + } } + @VisibleForTesting static final class IncrementalIndexRowComparator implements Comparator { @@ -1207,57 +1247,19 @@ public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) } } - private static boolean allNull(Object[] dims, int startPosition) - { - for (int i = startPosition; i < dims.length; i++) { - if (dims[i] != null) { - return false; - } - } - return true; - } - - public interface FactsHolder - { - /** - * @return the previous rowIndex associated with the specified key, or - * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. - */ - int getPriorIndex(IncrementalIndexRow key); - - long getMinTimeMillis(); - - long getMaxTimeMillis(); - - Iterator iterator(boolean descending); - - Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); - - Iterable keySet(); - - /** - * Get all {@link IncrementalIndexRow} to persist, ordered with {@link Comparator} - * - * @return - */ - Iterable persistIterable(); - - /** - * @return the previous rowIndex associated with the specified key, or - * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. - */ - int putIfAbsent(IncrementalIndexRow key, int rowIndex); - - void clear(); - } - - private final class LongMetricColumnSelector implements LongColumnSelector + private static final class LongMetricColumnSelector implements LongColumnSelector { + private final IncrementalIndexRowSelector rowSelector; private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public LongMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + public LongMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + int metricIndex + ) { + this.rowSelector = rowSelector; this.currEntry = currEntry; this.metricIndex = metricIndex; } @@ -1265,119 +1267,131 @@ public LongMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricI @Override public long getLong() { - assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricLongValue(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.getMetricLongValue(currEntry.get().getRowIndex(), metricIndex); } @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + public boolean isNull() { - inspector.visit("index", IncrementalIndex.this); + return rowSelector.isNull(currEntry.get().getRowIndex(), metricIndex); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + inspector.visit("index", rowSelector); } } - private final class ObjectMetricColumnSelector extends ObjectColumnSelector + private static final class FloatMetricColumnSelector implements FloatColumnSelector { + private final IncrementalIndexRowSelector rowSelector; private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - private Class classOfObject; - public ObjectMetricColumnSelector( - MetricDesc metricDesc, + public FloatMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, IncrementalIndexRowHolder currEntry, int metricIndex ) { this.currEntry = currEntry; + this.rowSelector = rowSelector; this.metricIndex = metricIndex; - classOfObject = ComplexMetrics.getSerdeForType(metricDesc.getType()).getObjectStrategy().getClazz(); } - @Nullable @Override - public Object getObject() + public float getFloat() { - return getMetricObjectValue(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.getMetricFloatValue(currEntry.get().getRowIndex(), metricIndex); } @Override - public Class classOfObject() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return classOfObject; + inspector.visit("index", rowSelector); } @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + public boolean isNull() { - inspector.visit("index", IncrementalIndex.this); + return rowSelector.isNull(currEntry.get().getRowIndex(), metricIndex); } } - private final class FloatMetricColumnSelector implements FloatColumnSelector + private static final class DoubleMetricColumnSelector implements DoubleColumnSelector { + private final IncrementalIndexRowSelector rowSelector; private final IncrementalIndexRowHolder currEntry; private final int metricIndex; - public FloatMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + public DoubleMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + int metricIndex + ) { this.currEntry = currEntry; + this.rowSelector = rowSelector; this.metricIndex = metricIndex; } @Override - public float getFloat() + public double getDouble() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricFloatValue(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.getMetricDoubleValue(currEntry.get().getRowIndex(), metricIndex); } @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + public boolean isNull() { - inspector.visit("index", IncrementalIndex.this); + return rowSelector.isNull(currEntry.get().getRowIndex(), metricIndex); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + inspector.visit("index", rowSelector); } } - private final class DoubleMetricColumnSelector implements DoubleColumnSelector + private static final class ObjectMetricColumnSelector extends ObjectColumnSelector { + private final IncrementalIndexRowSelector rowSelector; private final IncrementalIndexRowHolder currEntry; private final int metricIndex; + private final Class classOfObject; - public DoubleMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + public ObjectMetricColumnSelector( + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currEntry, + MetricDesc metricDesc + ) { this.currEntry = currEntry; - this.metricIndex = metricIndex; + this.rowSelector = rowSelector; + this.metricIndex = metricDesc.getIndex(); + this.classOfObject = ComplexMetrics.getSerdeForType(metricDesc.getType()).getObjectStrategy().getClazz(); } + @Nullable @Override - public double getDouble() + public Object getObject() { - assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricDoubleValue(currEntry.get().getRowIndex(), metricIndex); + return rowSelector.getMetricObjectValue(currEntry.get().getRowIndex(), metricIndex); } @Override - public boolean isNull() + public Class classOfObject() { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + return classOfObject; } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - inspector.visit("index", IncrementalIndex.this); + inspector.visit("index", rowSelector); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java index aae1c1335753..35736cc4975d 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java @@ -22,6 +22,7 @@ import it.unimi.dsi.fastutil.ints.IntIterator; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.MutableBitmap; +import org.apache.druid.error.DruidException; import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.IndexableAdapter; @@ -46,13 +47,15 @@ public class IncrementalIndexAdapter implements IndexableAdapter { private final Interval dataInterval; - private final IncrementalIndex index; + private final IncrementalIndexRowSelector index; private final Map accessors; + private final BitmapFactory bitmapFactory; - public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndex index, BitmapFactory bitmapFactory) + public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndexRowSelector index, BitmapFactory bitmapFactory) { this.dataInterval = dataInterval; this.index = index; + this.bitmapFactory = bitmapFactory; final List dimensions = index.getDimensions(); accessors = dimensions @@ -73,7 +76,7 @@ public IncrementalIndexAdapter(Interval dataInterval, IncrementalIndex index, Bi * a null value. */ private void processRows( - IncrementalIndex index, + IncrementalIndexRowSelector index, BitmapFactory bitmapFactory, List dimensions ) @@ -112,7 +115,7 @@ public Interval getDataInterval() @Override public int getNumRows() { - return index.size(); + return index.numRows(); } @Override @@ -182,6 +185,16 @@ public TransformableRowIterator getRows() return new IncrementalIndexRowIterator(index); } + @Override + public IndexableAdapter getProjectionAdapter(String projection) + { + if (index instanceof IncrementalIndex) { + IncrementalIndexRowSelector projectionSelector = ((IncrementalIndex) index).getProjection(projection); + return new IncrementalIndexAdapter(dataInterval, projectionSelector, bitmapFactory); + } + throw DruidException.defensive("projection inception"); + } + @Override public BitmapValues getBitmapValues(String dimension, int index) { @@ -225,7 +238,10 @@ public ColumnFormat getFormat(String column) @Override public Metadata getMetadata() { - return index.getMetadata(); + if (index instanceof IncrementalIndex) { + return ((IncrementalIndex) index).getMetadata(); + } + throw DruidException.defensive("cannot get metadata of projection"); } static class MutableBitmapValues implements BitmapValues diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java index 86e8c6690c2d..68cc263950df 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexColumnSelectorFactory.java @@ -43,19 +43,19 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, RowIdSupplier { private final ColumnInspector snapshotColumnInspector; - private final IncrementalIndex index; private final VirtualColumns virtualColumns; private final Order timeOrder; private final IncrementalIndexRowHolder rowHolder; + private final IncrementalIndexRowSelector rowSelector; IncrementalIndexColumnSelectorFactory( - IncrementalIndex index, + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder rowHolder, VirtualColumns virtualColumns, - Order timeOrder, - IncrementalIndexRowHolder rowHolder + Order timeOrder ) { - this.index = index; + this.rowSelector = rowSelector; this.virtualColumns = virtualColumns; this.timeOrder = timeOrder; this.rowHolder = rowHolder; @@ -65,7 +65,7 @@ class IncrementalIndexColumnSelectorFactory implements ColumnSelectorFactory, Ro @Override public ColumnCapabilities getColumnCapabilities(String column) { - return IncrementalIndexCursorFactory.snapshotColumnCapabilities(index, column); + return IncrementalIndexCursorFactory.snapshotColumnCapabilities(rowSelector, column); } }; } @@ -85,15 +85,11 @@ private DimensionSelector makeDimensionSelectorUndecorated(DimensionSpec dimensi final String dimension = dimensionSpec.getDimension(); final ExtractionFn extractionFn = dimensionSpec.getExtractionFn(); - if (dimension.equals(ColumnHolder.TIME_COLUMN_NAME) && timeOrder != Order.NONE) { - return new SingleScanTimeDimensionSelector( - makeColumnValueSelector(dimension), - extractionFn, - timeOrder - ); + if (isTimeColumn(dimension) && timeOrder != Order.NONE) { + return new SingleScanTimeDimensionSelector(makeColumnValueSelector(dimension), extractionFn, timeOrder); } - final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimensionSpec.getDimension()); + final IncrementalIndex.DimensionDesc dimensionDesc = rowSelector.getDimension(dimensionSpec.getDimension()); if (dimensionDesc == null) { // not a dimension, column may be a metric ColumnCapabilities capabilities = getColumnCapabilities(dimension); @@ -122,19 +118,17 @@ public ColumnValueSelector makeColumnValueSelector(String columnName) if (virtualColumns.exists(columnName)) { return virtualColumns.makeColumnValueSelector(columnName, this); } - - if (columnName.equals(ColumnHolder.TIME_COLUMN_NAME)) { + if (isTimeColumn(columnName)) { return rowHolder; } - final Integer dimIndex = index.getDimensionIndex(columnName); - if (dimIndex != null) { - final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(columnName); + final IncrementalIndex.DimensionDesc dimensionDesc = rowSelector.getDimension(columnName); + if (dimensionDesc != null) { final DimensionIndexer indexer = dimensionDesc.getIndexer(); return indexer.makeColumnValueSelector(rowHolder, dimensionDesc); } - return index.makeMetricColumnValueSelector(columnName, rowHolder); + return IncrementalIndex.makeMetricColumnValueSelector(rowSelector, rowHolder, columnName); } @Override @@ -142,6 +136,9 @@ public ColumnValueSelector makeColumnValueSelector(String columnName) public ColumnCapabilities getColumnCapabilities(String columnName) { // Use snapshotColumnInspector instead of index.getCapabilities (see note in IncrementalIndexStorageAdapater) + if (isTimeColumn(columnName)) { + return virtualColumns.getColumnCapabilitiesWithFallback(snapshotColumnInspector, ColumnHolder.TIME_COLUMN_NAME); + } return virtualColumns.getColumnCapabilitiesWithFallback(snapshotColumnInspector, columnName); } @@ -157,4 +154,9 @@ public long getRowId() { return rowHolder.get().getRowIndex(); } + + private boolean isTimeColumn(String columnName) + { + return ColumnHolder.TIME_COLUMN_NAME.equals(columnName); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java index e034f820dfbf..5f42fbaaf474 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactory.java @@ -20,6 +20,8 @@ package org.apache.druid.segment.incremental; import com.google.common.collect.Iterables; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.CursorHolder; @@ -28,8 +30,10 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.projections.QueryableProjection; import javax.annotation.Nullable; +import java.util.List; public class IncrementalIndexCursorFactory implements CursorFactory { @@ -77,7 +81,35 @@ public IncrementalIndexCursorFactory(IncrementalIndex index) @Override public CursorHolder makeCursorHolder(CursorBuildSpec spec) { - return new IncrementalIndexCursorHolder(index, spec); + final QueryableProjection projection = index.getProjection(spec); + if (projection == null) { + return new IncrementalIndexCursorHolder(index, spec); + } else { + // currently we only have aggregated projections, so isPreAggregated is always true + return new IncrementalIndexCursorHolder( + projection.getRowSelector(), + projection.getCursorBuildSpec() + ) + { + @Override + public ColumnSelectorFactory makeSelectorFactory(CursorBuildSpec buildSpec, IncrementalIndexRowHolder currEntry) + { + return projection.wrapColumnSelectorFactory(super.makeSelectorFactory(buildSpec, currEntry)); + } + + @Override + public boolean isPreAggregated() + { + return true; + } + + @Override + public List getAggregatorsForPreAggregated() + { + return projection.getCursorBuildSpec().getAggregators(); + } + }; + } } @Override @@ -99,9 +131,9 @@ public ColumnCapabilities getColumnCapabilities(String column) return snapshotColumnCapabilities(index, column); } - static ColumnCapabilities snapshotColumnCapabilities(IncrementalIndex index, String column) + static ColumnCapabilities snapshotColumnCapabilities(IncrementalIndexRowSelector selector, String column) { - IncrementalIndex.DimensionDesc desc = index.getDimension(column); + IncrementalIndex.DimensionDesc desc = selector.getDimension(column); // nested column indexer is a liar, and behaves like any type if it only processes unnested literals of a single // type, so force it to use nested column type if (desc != null && desc.getIndexer() instanceof NestedDataColumnIndexerV4) { @@ -122,7 +154,7 @@ static ColumnCapabilities snapshotColumnCapabilities(IncrementalIndex index, Str // multi-valuedness at cursor creation time, instead of the latest state, and getSnapshotColumnCapabilities could // be removed. return ColumnCapabilitiesImpl.snapshot( - index.getColumnCapabilities(column), + selector.getColumnCapabilities(column), COERCE_LOGIC ); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java index 02c09398d8e5..43b887ef6332 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexCursorHolder.java @@ -23,52 +23,48 @@ import org.apache.druid.query.BaseQuery; import org.apache.druid.query.Order; import org.apache.druid.query.OrderBy; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.CursorHolder; import org.apache.druid.segment.Cursors; -import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.filter.ValueMatchers; -import org.joda.time.Interval; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -import javax.annotation.Nullable; -import java.util.Collections; import java.util.Iterator; import java.util.List; public class IncrementalIndexCursorHolder implements CursorHolder { - private final IncrementalIndex index; + private final IncrementalIndexRowSelector rowSelector; private final CursorBuildSpec spec; private final List ordering; public IncrementalIndexCursorHolder( - IncrementalIndex index, + IncrementalIndexRowSelector rowSelector, CursorBuildSpec spec ) { - this.index = index; + this.rowSelector = rowSelector; this.spec = spec; - if (index.timePosition == 0) { + List ordering = rowSelector.getOrdering(); + if (Cursors.getTimeOrdering(ordering) != Order.NONE) { if (Cursors.preferDescendingTimeOrdering(spec)) { this.ordering = Cursors.descendingTimeOrder(); } else { this.ordering = Cursors.ascendingTimeOrder(); } } else { - // In principle, we could report a sort order here for certain types of fact holders; for example the - // RollupFactsHolder would be sorted by dimensions. However, this is left for future work. - this.ordering = Collections.emptyList(); + this.ordering = ordering; } } @Override public Cursor asCursor() { - if (index.isEmpty()) { + if (rowSelector.isEmpty()) { return null; } @@ -76,13 +72,13 @@ public Cursor asCursor() spec.getQueryMetrics().vectorized(false); } - + IncrementalIndexRowHolder currentRow = new IncrementalIndexRowHolder(); return new IncrementalIndexCursor( - index, - spec.getVirtualColumns(), - Cursors.getTimeOrdering(ordering), - spec.getFilter(), - spec.getInterval() + rowSelector, + currentRow, + makeSelectorFactory(spec, currentRow), + spec, + getTimeOrder(ordering) ); } @@ -92,46 +88,50 @@ public List getOrdering() return ordering; } + public ColumnSelectorFactory makeSelectorFactory(CursorBuildSpec buildSpec, IncrementalIndexRowHolder currEntry) + { + return new IncrementalIndexColumnSelectorFactory( + rowSelector, + currEntry, + buildSpec.getVirtualColumns(), + getTimeOrder() + ); + } + static class IncrementalIndexCursor implements Cursor { - private IncrementalIndexRowHolder currEntry; + private final Iterable cursorIterable; + private final IncrementalIndexRowHolder currEntry; private final ColumnSelectorFactory columnSelectorFactory; private final ValueMatcher filterMatcher; private final int maxRowIndex; - private final IncrementalIndex.FactsHolder facts; + @MonotonicNonNull private Iterator baseIter; - private Iterable cursorIterable; - private boolean emptyRange; private int numAdvanced; private boolean done; IncrementalIndexCursor( - IncrementalIndex index, - VirtualColumns virtualColumns, - Order timeOrder, - @Nullable Filter filter, - Interval actualInterval + IncrementalIndexRowSelector rowSelector, + IncrementalIndexRowHolder currentRow, + ColumnSelectorFactory selectorFactory, + CursorBuildSpec buildSpec, + Order timeOrder ) { - currEntry = new IncrementalIndexRowHolder(); - columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( - index, - virtualColumns, - timeOrder, - currEntry - ); + currEntry = currentRow; + columnSelectorFactory = selectorFactory; // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 - maxRowIndex = index.getLastRowIndex(); - filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); + maxRowIndex = rowSelector.getLastRowIndex(); numAdvanced = -1; - facts = index.getFacts(); - cursorIterable = facts.timeRangeIterable( + + cursorIterable = rowSelector.getFacts().timeRangeIterable( timeOrder == Order.DESCENDING, - actualInterval.getStartMillis(), - actualInterval.getEndMillis() + buildSpec.getInterval().getStartMillis(), + buildSpec.getInterval().getEndMillis() ); - emptyRange = !cursorIterable.iterator().hasNext(); - + filterMatcher = buildSpec.getFilter() == null + ? ValueMatchers.allTrue() + : buildSpec.getFilter().makeMatcher(columnSelectorFactory); reset(); } @@ -152,7 +152,7 @@ public void advance() while (baseIter.hasNext()) { BaseQuery.checkInterrupted(); - IncrementalIndexRow entry = baseIter.next(); + final IncrementalIndexRow entry = baseIter.next(); if (beyondMaxRowIndex(entry.getRowIndex())) { continue; } @@ -236,7 +236,7 @@ public void reset() numAdvanced++; } - done = !foundMatched && (emptyRange || !baseIter.hasNext()); + done = !foundMatched; } private boolean beyondMaxRowIndex(int rowIndex) @@ -247,4 +247,13 @@ private boolean beyondMaxRowIndex(int rowIndex) return rowIndex > maxRowIndex; } } + + private static Order getTimeOrder(List ordering) + { + if (!ordering.isEmpty() && ColumnHolder.TIME_COLUMN_NAME.equals(ordering.get(0).getColumnName())) { + return ordering.get(0).getOrder(); + } else { + return Order.NONE; + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java index c015206a9da2..d8b6d46105aa 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexPhysicalSegmentInspector.java @@ -134,6 +134,6 @@ public ColumnCapabilities getColumnCapabilities(String column) @Override public int getNumRows() { - return index.size(); + return index.numRows(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index 89e94961f6b2..2e817b993ce0 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java @@ -144,6 +144,8 @@ public Object apply(@Nullable Object input) { if (input == null || (input.getClass().isArray() && Array.getLength(input) == 0)) { return Collections.singletonList("null"); + } else if (input instanceof int[]) { + return Arrays.toString((int[]) input); } return Collections.singletonList(input); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java index 15909ac1b527..b0d5cbb45b74 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java @@ -49,7 +49,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator private final RowPointer currentRowPointer; private final TimeAndDimsPointer markedRowPointer; - IncrementalIndexRowIterator(IncrementalIndex incrementalIndex) + IncrementalIndexRowIterator(IncrementalIndexRowSelector incrementalIndex) { this.timeAndDimsIterator = incrementalIndex.getFacts().persistIterable().iterator(); this.currentRowPointer = makeRowPointer(incrementalIndex, currentRowHolder, currentRowNumCounter); @@ -59,7 +59,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator } private static RowPointer makeRowPointer( - IncrementalIndex incrementalIndex, + IncrementalIndexRowSelector incrementalIndex, IncrementalIndexRowHolder rowHolder, RowNumCounter rowNumCounter ) @@ -67,9 +67,9 @@ private static RowPointer makeRowPointer( ColumnSelectorFactory columnSelectorFactory = new IncrementalIndexColumnSelectorFactory( incrementalIndex, + rowHolder, VirtualColumns.EMPTY, - incrementalIndex.timePosition == 0 ? Order.ASCENDING : Order.NONE, - rowHolder + incrementalIndex.getTimePosition() == 0 ? Order.ASCENDING : Order.NONE ); ColumnValueSelector[] dimensionSelectors = incrementalIndex .getDimensions() @@ -92,7 +92,7 @@ private static RowPointer makeRowPointer( return new RowPointer( rowHolder, - incrementalIndex.timePosition, + incrementalIndex.getTimePosition(), dimensionSelectors, dimensionHandlers, metricSelectors, diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java new file mode 100644 index 000000000000..09e877190f3d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowSelector.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.incremental; + +import org.apache.druid.query.OrderBy; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.column.ColumnFormat; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * Interface that abstracts selecting data from a {@link FactsHolder} + */ +public interface IncrementalIndexRowSelector extends ColumnInspector +{ + /** + * Returns list of {@link IncrementalIndex.DimensionDesc} for the row selector + */ + List getDimensions(); + + /** + * Returns list of dimension names for the row selector, optionally including the time column. If time is included, + * the order columns appear in this list will match {@link #getOrdering()} + */ + List getDimensionNames(boolean includeTime); + + /** + * Returns list of all metric column names for the row selector + */ + List getMetricNames(); + + /** + * get {@link IncrementalIndex.DimensionDesc} for the specified column, if available, which provides access to things + * like {@link org.apache.druid.segment.DimensionIndexer} and {@link org.apache.druid.segment.DimensionHandler} as + * well as column capabilities and position within the row + */ + @Nullable + IncrementalIndex.DimensionDesc getDimension(String columnName); + + /** + * Get {@link IncrementalIndex.MetricDesc} which provides column capabilities and position in the aggregators section + * of the row + */ + @Nullable + IncrementalIndex.MetricDesc getMetric(String s); + + /** + * Get {@link ColumnFormat} for a dimension, metrics, or time column, or null if the column does not exist + */ + @Nullable + ColumnFormat getColumnFormat(String columnName); + + /** + * Ordering for the data in the facts table + */ + List getOrdering(); + + /** + * Position of the time column in {@link #getOrdering()} + */ + int getTimePosition(); + + /** + * Are there any {@link IncrementalIndexRow} stored in the {@link FactsHolder}? + */ + boolean isEmpty(); + + /** + * Number of rows in {@link FactsHolder} + */ + int numRows(); + + /** + * Get the {@link FactsHolder} containing all of the {@link IncrementalIndexRow} backing this selector + */ + FactsHolder getFacts(); + + /** + * Highest value {@link IncrementalIndexRow#getRowIndex()} available in this selector. Note that these values do not + * reflect the position of the row in the {@link FactsHolder}, rather just the order in which they were processed + */ + int getLastRowIndex(); + + /** + * @param rowOffset row to get float aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return float value of the metric + */ + float getMetricFloatValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to get long aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return long value of the aggregator for this row + */ + long getMetricLongValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to get double aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return double value of the aggregator for this row + */ + double getMetricDoubleValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to get long aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return long value of the aggregator for this row + */ + @Nullable + Object getMetricObjectValue(int rowOffset, int aggOffset); + + /** + * @param rowOffset row to check for a aggregator value + * @param aggOffset position of the aggregator in the aggregators array of the data schema + * @return is the value null for this row? + */ + boolean isNull(int rowOffset, int aggOffset); +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java index 8866c413dc74..fdeda126a685 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexSchema.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.data.input.impl.TimestampSpec; @@ -28,6 +29,8 @@ import org.apache.druid.segment.VirtualColumns; import javax.annotation.Nullable; +import java.util.Collections; +import java.util.List; /** */ @@ -46,6 +49,8 @@ public static IncrementalIndexSchema.Builder builder() private final AggregatorFactory[] metrics; private final boolean rollup; + private final List projections; + public IncrementalIndexSchema( long minTimestamp, TimestampSpec timestampSpec, @@ -53,7 +58,8 @@ public IncrementalIndexSchema( VirtualColumns virtualColumns, DimensionsSpec dimensionsSpec, AggregatorFactory[] metrics, - boolean rollup + boolean rollup, + List projections ) { this.minTimestamp = minTimestamp; @@ -63,6 +69,7 @@ public IncrementalIndexSchema( this.dimensionsSpec = dimensionsSpec; this.metrics = metrics; this.rollup = rollup; + this.projections = projections; } public long getMinTimestamp() @@ -100,6 +107,11 @@ public boolean isRollup() return rollup; } + public List getProjections() + { + return projections; + } + public static class Builder { private long minTimestamp; @@ -109,6 +121,7 @@ public static class Builder private DimensionsSpec dimensionsSpec; private AggregatorFactory[] metrics; private boolean rollup; + private List projections; public Builder() { @@ -118,6 +131,7 @@ public Builder() this.dimensionsSpec = DimensionsSpec.EMPTY; this.metrics = new AggregatorFactory[]{}; this.rollup = true; + this.projections = Collections.emptyList(); } public Builder withMinTimestamp(long minTimestamp) @@ -176,6 +190,12 @@ public Builder withRollup(boolean rollup) return this; } + public Builder withProjections(@Nullable List projections) + { + this.projections = projections == null ? Collections.emptyList() : projections; + return this; + } + public IncrementalIndexSchema build() { return new IncrementalIndexSchema( @@ -185,7 +205,8 @@ public IncrementalIndexSchema build() virtualColumns, dimensionsSpec, metrics, - rollup + rollup, + projections ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnHeapAggregateProjection.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnHeapAggregateProjection.java new file mode 100644 index 000000000000..75650a1c1827 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnHeapAggregateProjection.java @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.incremental; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.EncodedKeyComponent; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnFormat; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nullable; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Projection of {@link OnheapIncrementalIndex} for {@link org.apache.druid.data.input.impl.AggregateProjectionSpec} + */ +public class OnHeapAggregateProjection implements IncrementalIndexRowSelector +{ + private final AggregateProjectionMetadata.Schema projectionSchema; + private final List dimensions; + private final int[] parentDimensionIndex; + private final AggregatorFactory[] aggregatorFactories; + private final Map dimensionsMap; + private final Map aggregatorsMap; + private final Map columnFormats; + private final FactsHolder factsHolder; + private final IncrementalIndex.InputRowHolder inputRowHolder = new IncrementalIndex.InputRowHolder(); + private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); + private final ColumnSelectorFactory virtualSelectorFactory; + private final Map aggSelectors; + private final boolean useMaxMemoryEstimates; + private final long maxBytesPerRowForAggregators; + private final long minTimestamp; + private final AtomicInteger rowCounter = new AtomicInteger(0); + private final AtomicInteger numEntries = new AtomicInteger(0); + + public OnHeapAggregateProjection( + AggregateProjectionMetadata.Schema schema, + List dimensions, + Map dimensionsMap, + int[] parentDimensionIndex, + long minTimestamp, + boolean useMaxMemoryEstimates, + long maxBytesPerRowForAggregators + ) + { + this.projectionSchema = schema; + this.dimensions = dimensions; + this.parentDimensionIndex = parentDimensionIndex; + this.dimensionsMap = dimensionsMap; + this.minTimestamp = minTimestamp; + final IncrementalIndex.IncrementalIndexRowComparator rowComparator = new IncrementalIndex.IncrementalIndexRowComparator( + projectionSchema.getTimeColumnPosition() < 0 ? dimensions.size() : projectionSchema.getTimeColumnPosition(), + dimensions + ); + this.factsHolder = new OnheapIncrementalIndex.RollupFactsHolder( + rowComparator, + dimensions, + projectionSchema.getTimeColumnPosition() == 0 + ); + this.useMaxMemoryEstimates = useMaxMemoryEstimates; + this.maxBytesPerRowForAggregators = maxBytesPerRowForAggregators; + + this.virtualSelectorFactory = new OnheapIncrementalIndex.CachingColumnSelectorFactory( + IncrementalIndex.makeColumnSelectorFactory(schema.getVirtualColumns(), inputRowHolder, null) + ); + this.aggSelectors = new LinkedHashMap<>(); + this.aggregatorsMap = new LinkedHashMap<>(); + this.aggregatorFactories = new AggregatorFactory[schema.getAggregators().length]; + this.columnFormats = new LinkedHashMap<>(); + for (IncrementalIndex.DimensionDesc dimension : dimensions) { + if (dimension.getName().equals(projectionSchema.getTimeColumnName())) { + columnFormats.put( + dimension.getName(), + new CapabilitiesBasedFormat(ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG)) + ); + } else { + columnFormats.put(dimension.getName(), dimension.getIndexer().getFormat()); + } + } + int i = 0; + for (AggregatorFactory agg : schema.getAggregators()) { + IncrementalIndex.MetricDesc metricDesc = new IncrementalIndex.MetricDesc(aggregatorsMap.size(), agg); + aggregatorsMap.put(metricDesc.getName(), metricDesc); + columnFormats.put(metricDesc.getName(), new CapabilitiesBasedFormat(metricDesc.getCapabilities())); + final ColumnSelectorFactory factory; + if (agg.getIntermediateType().is(ValueType.COMPLEX)) { + factory = new OnheapIncrementalIndex.CachingColumnSelectorFactory( + IncrementalIndex.makeColumnSelectorFactory(VirtualColumns.EMPTY, inputRowHolder, agg) + ); + } else { + factory = virtualSelectorFactory; + } + aggSelectors.put(agg.getName(), factory); + aggregatorFactories[i++] = agg; + } + } + + /** + * Add row to projection {@link #factsHolder}, updating totalSizeInBytes estimate + */ + public void addToFacts( + IncrementalIndexRow key, + InputRow inputRow, + List parseExceptionMessages, + AtomicLong totalSizeInBytes + ) + { + inputRowHolder.set(inputRow); + final Object[] projectionDims = new Object[dimensions.size()]; + for (int i = 0; i < projectionDims.length; i++) { + int parentDimIndex = parentDimensionIndex[i]; + if (parentDimIndex < 0) { + IncrementalIndex.DimensionDesc desc = dimensions.get(i); + final ColumnValueSelector virtualSelector = virtualSelectorFactory.makeColumnValueSelector(desc.getName()); + EncodedKeyComponent k = desc.getIndexer().processRowValsToUnsortedEncodedKeyComponent( + virtualSelector.getObject(), + false + ); + projectionDims[i] = k.getComponent(); + totalSizeInBytes.addAndGet(k.getEffectiveSizeBytes()); + } else { + projectionDims[i] = key.dims[parentDimensionIndex[i]]; + } + } + final IncrementalIndexRow subKey = new IncrementalIndexRow( + projectionSchema.getTimeColumnName() != null + ? projectionSchema.getGranularity().bucketStart(DateTimes.utc(key.getTimestamp())).getMillis() + : minTimestamp, + projectionDims, + dimensions + ); + + final int priorIndex = factsHolder.getPriorIndex(subKey); + + final Aggregator[] aggs; + if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { + aggs = aggregators.get(priorIndex); + long aggForProjectionSizeDelta = OnheapIncrementalIndex.doAggregate( + aggregatorFactories, + aggs, + inputRowHolder, + parseExceptionMessages, + useMaxMemoryEstimates, + false + ); + totalSizeInBytes.addAndGet(useMaxMemoryEstimates ? 0 : aggForProjectionSizeDelta); + } else { + aggs = new Aggregator[aggregatorFactories.length]; + long aggSizeForProjectionRow = factorizeAggs(aggregatorFactories, aggs); + aggSizeForProjectionRow += OnheapIncrementalIndex.doAggregate( + aggregatorFactories, + aggs, + inputRowHolder, + parseExceptionMessages, + useMaxMemoryEstimates, + false + ); + final long estimatedSizeOfAggregators = + useMaxMemoryEstimates ? maxBytesPerRowForAggregators : aggSizeForProjectionRow; + final long projectionRowSize = key.estimateBytesInMemory() + + estimatedSizeOfAggregators + + OnheapIncrementalIndex.ROUGH_OVERHEAD_PER_MAP_ENTRY; + totalSizeInBytes.addAndGet(useMaxMemoryEstimates ? 0 : projectionRowSize); + numEntries.incrementAndGet(); + } + final int rowIndex = rowCounter.getAndIncrement(); + aggregators.put(rowIndex, aggs); + factsHolder.putIfAbsent(subKey, rowIndex); + } + + @Override + public FactsHolder getFacts() + { + return factsHolder; + } + + @Override + public List getDimensions() + { + return dimensions; + } + + @Override + public List getMetricNames() + { + return ImmutableList.copyOf(aggregatorsMap.keySet()); + } + + @Override + public IncrementalIndex.DimensionDesc getDimension(String columnName) + { + return dimensionsMap.get(columnName); + } + + @Override + public IncrementalIndex.MetricDesc getMetric(String columnName) + { + return aggregatorsMap.get(columnName); + } + + @Override + public List getOrdering() + { + // return ordering with projection time column substituted with __time so query engines can treat it equivalently + return projectionSchema.getOrderingWithTimeColumnSubstitution(); + } + + @Override + public int getTimePosition() + { + return projectionSchema.getTimeColumnPosition(); + } + + @Override + public boolean isEmpty() + { + return rowCounter.get() == 0; + } + + @Override + public int getLastRowIndex() + { + return rowCounter.get(); + } + + @Override + public float getMetricFloatValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].getFloat(); + } + + @Override + public long getMetricLongValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].getLong(); + } + + @Override + public double getMetricDoubleValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].getDouble(); + } + + @Nullable + @Override + public Object getMetricObjectValue(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].get(); + } + + @Override + public boolean isNull(int rowOffset, int aggOffset) + { + return aggregators.get(rowOffset)[aggOffset].isNull(); + } + + @Override + public ColumnFormat getColumnFormat(String columnName) + { + return columnFormats.get(columnName); + } + + @Override + public int numRows() + { + return numEntries.get(); + } + + @Override + public List getDimensionNames(boolean includeTime) + { + synchronized (dimensionsMap) { + if (includeTime && projectionSchema.getTimeColumnName() != null) { + final ImmutableList.Builder listBuilder = + ImmutableList.builderWithExpectedSize(dimensionsMap.size() + 1); + int i = 0; + if (i == projectionSchema.getTimeColumnPosition()) { + listBuilder.add(projectionSchema.getTimeColumnName()); + } + for (String dimName : dimensionsMap.keySet()) { + listBuilder.add(dimName); + i++; + if (i == projectionSchema.getTimeColumnPosition()) { + listBuilder.add(projectionSchema.getTimeColumnName()); + } + } + return listBuilder.build(); + } else { + return ImmutableList.copyOf(dimensionsMap.keySet()); + } + } + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + if (ColumnHolder.TIME_COLUMN_NAME.equals(column) || Objects.equals(column, projectionSchema.getTimeColumnName())) { + return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG).setHasNulls(false); + } + if (dimensionsMap.containsKey(column)) { + return dimensionsMap.get(column).getCapabilities(); + } + if (aggregatorsMap.containsKey(column)) { + return aggregatorsMap.get(column).getCapabilities(); + } + return null; + } + + public Map getDimensionsMap() + { + return dimensionsMap; + } + + public AggregateProjectionMetadata toMetadata() + { + return new AggregateProjectionMetadata(projectionSchema, numEntries.get()); + } + + private long factorizeAggs(AggregatorFactory[] aggregatorFactories, Aggregator[] aggs) + { + long totalInitialSizeBytes = 0L; + final long aggReferenceSize = Long.BYTES; + for (int i = 0; i < aggregatorFactories.length; i++) { + final AggregatorFactory agg = aggregatorFactories[i]; + // Creates aggregators to aggregate from input into output fields + if (useMaxMemoryEstimates) { + aggs[i] = agg.factorize(aggSelectors.get(agg.getName())); + } else { + AggregatorAndSize aggregatorAndSize = agg.factorizeWithSize(aggSelectors.get(agg.getName())); + aggs[i] = aggregatorAndSize.getAggregator(); + totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes(); + totalInitialSizeBytes += aggReferenceSize; + } + } + return totalInitialSizeBytes; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java index b5e580f44f2f..d6c67a2fcaa4 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapIncrementalIndex.java @@ -24,9 +24,13 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; +import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.error.DruidException; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; @@ -37,19 +41,27 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.Metadata; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.projections.Projections; +import org.apache.druid.segment.projections.QueryableProjection; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Comparator; import java.util.Deque; @@ -58,6 +70,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.SortedSet; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; @@ -66,6 +79,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -86,7 +100,8 @@ public class OnheapIncrementalIndex extends IncrementalIndex /** * overhead per {@link ConcurrentSkipListMap.Node} object in facts table */ - private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; + static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; + private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); private final FactsHolder facts; private final AtomicInteger indexIncrement = new AtomicInteger(0); @@ -135,6 +150,9 @@ public class OnheapIncrementalIndex extends IncrementalIndex @Nullable private String outOfRowsReason = null; + private final SortedSet aggregateProjections; + private final HashMap projections; + OnheapIncrementalIndex( IncrementalIndexSchema incrementalIndexSchema, int maxRowCount, @@ -155,9 +173,83 @@ public class OnheapIncrementalIndex extends IncrementalIndex } else { this.facts = new PlainNonTimeOrderedFactsHolder(dimsComparator()); } - maxBytesPerRowForAggregators = + this.maxBytesPerRowForAggregators = useMaxMemoryEstimates ? getMaxBytesPerRowForAggregators(incrementalIndexSchema) : 0; this.useMaxMemoryEstimates = useMaxMemoryEstimates; + + this.aggregateProjections = new ObjectAVLTreeSet<>(AggregateProjectionMetadata.COMPARATOR); + this.projections = new HashMap<>(); + initializeProjections(incrementalIndexSchema, useMaxMemoryEstimates); + } + + private void initializeProjections(IncrementalIndexSchema incrementalIndexSchema, boolean useMaxMemoryEstimates) + { + for (AggregateProjectionSpec projectionSpec : incrementalIndexSchema.getProjections()) { + // initialize them all with 0 rows + AggregateProjectionMetadata.Schema schema = projectionSpec.toMetadataSchema(); + aggregateProjections.add(new AggregateProjectionMetadata(schema, 0)); + final List descs = new ArrayList<>(); + // mapping of position in descs on the projection to position in the parent incremental index. Like the parent + // incremental index, the time (or time-like) column does not have a dimension descriptor and is specially + // handled as the timestamp of the row. Unlike the parent incremental index, an aggregating projection will + // always have its time-like column in the grouping columns list, so its position in this array specifies -1 + final int[] parentDimIndex = new int[projectionSpec.getGroupingColumns().size()]; + Arrays.fill(parentDimIndex, -1); + int i = 0; + final Map dimensionsMap = new HashMap<>(); + for (DimensionSchema dimension : projectionSpec.getGroupingColumns()) { + if (dimension.getName().equals(schema.getTimeColumnName())) { + continue; + } + final DimensionDesc parent = getDimension(dimension.getName()); + if (parent == null) { + // this dimension only exists in the child, it needs its own handler + final DimensionDesc childOnly = new DimensionDesc( + i++, + dimension.getName(), + dimension.getDimensionHandler(), + useMaxMemoryEstimates + ); + descs.add(childOnly); + dimensionsMap.put(dimension.getName(), childOnly); + } else { + if (!dimension.getColumnType().equals(parent.getCapabilities().toColumnType())) { + // special handle auto column schema, who reports type as json in schema, but indexer reports whatever + // type it has seen, which is string at this stage + boolean allowAuto = ColumnType.NESTED_DATA.equals(dimension.getColumnType()) && + parent.getIndexer() instanceof AutoTypeColumnIndexer; + InvalidInput.conditionalException( + allowAuto, + "projection[%s] contains dimension[%s] with different type[%s] than type[%s] in base table", + projectionSpec.getName(), + dimension.getName(), + dimension.getColumnType(), + parent.getCapabilities().toColumnType() + ); + } + // make a new DimensionDesc from the child, containing all of the parents stuff but with the childs position + final DimensionDesc child = new DimensionDesc( + i++, + parent.getName(), + parent.getHandler(), + parent.getIndexer() + ); + descs.add(child); + dimensionsMap.put(dimension.getName(), child); + parentDimIndex[child.getIndex()] = parent.getIndex(); + } + } + final OnHeapAggregateProjection projection = new OnHeapAggregateProjection( + projectionSpec.toMetadataSchema(), + descs, + dimensionsMap, + parentDimIndex, + incrementalIndexSchema.getMinTimestamp(), + this.useMaxMemoryEstimates, + this.maxBytesPerRowForAggregators + ); + projections.put(projectionSpec.getName(), projection); + } } /** @@ -202,6 +294,19 @@ public FactsHolder getFacts() return facts; } + @Override + public Metadata getMetadata() + { + if (aggregateProjections.isEmpty()) { + return super.getMetadata(); + } + final List projectionMetadata = projections.values() + .stream() + .map(OnHeapAggregateProjection::toMetadata) + .collect(Collectors.toList()); + return super.getMetadata().withProjections(projectionMetadata); + } + @Override protected void initAggs( final AggregatorFactory[] metrics, @@ -252,14 +357,21 @@ protected AddToFactsResult addToFacts( ) throws IndexSizeExceededException { final List parseExceptionMessages = new ArrayList<>(); + final AtomicLong totalSizeInBytes = getBytesInMemory(); + + // add to projections first so if one is chosen by queries the data will always be ahead of the base table since + // rows are not added atomically to all facts holders at once + for (OnHeapAggregateProjection projection : projections.values()) { + projection.addToFacts(key, inputRowHolder.getRow(), parseExceptionMessages, totalSizeInBytes); + } + final int priorIndex = facts.getPriorIndex(key); Aggregator[] aggs; final AggregatorFactory[] metrics = getMetrics(); final AtomicInteger numEntries = getNumEntries(); - final AtomicLong totalSizeInBytes = getBytesInMemory(); if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { - aggs = concurrentGet(priorIndex); + aggs = aggregators.get(priorIndex); long aggSizeDelta = doAggregate(metrics, aggs, inputRowHolder, parseExceptionMessages); totalSizeInBytes.addAndGet(useMaxMemoryEstimates ? 0 : aggSizeDelta); } else { @@ -272,7 +384,7 @@ protected AddToFactsResult addToFacts( aggSizeForRow += doAggregate(metrics, aggs, inputRowHolder, parseExceptionMessages); final int rowIndex = indexIncrement.getAndIncrement(); - concurrentSet(rowIndex, aggs); + aggregators.put(rowIndex, aggs); // Last ditch sanity checks if ((numEntries.get() >= maxRowCount || totalSizeInBytes.get() >= maxBytesInMemory) @@ -335,11 +447,10 @@ private long factorizeAggs( } // Creates aggregators to combine already aggregated field if (preserveExistingMetrics) { + AggregatorFactory combiningAgg = agg.getCombiningFactory(); if (useMaxMemoryEstimates) { - AggregatorFactory combiningAgg = agg.getCombiningFactory(); aggs[i + metrics.length] = combiningAgg.factorize(combiningAggSelectors.get(combiningAgg.getName())); } else { - AggregatorFactory combiningAgg = agg.getCombiningFactory(); AggregatorAndSize aggregatorAndSize = combiningAgg.factorizeWithSize(combiningAggSelectors.get(combiningAgg.getName())); aggs[i + metrics.length] = aggregatorAndSize.getAggregator(); totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes(); @@ -363,6 +474,18 @@ private long doAggregate( InputRowHolder inputRowHolder, List parseExceptionsHolder ) + { + return doAggregate(metrics, aggs, inputRowHolder, parseExceptionsHolder, useMaxMemoryEstimates, preserveExistingMetrics); + } + + static long doAggregate( + AggregatorFactory[] metrics, + Aggregator[] aggs, + InputRowHolder inputRowHolder, + List parseExceptionsHolder, + boolean useMaxMemoryEstimates, + boolean preserveExistingMetrics + ) { long totalIncrementalBytes = 0L; for (int i = 0; i < metrics.length; i++) { @@ -418,21 +541,29 @@ private void closeAggregators() } } - protected Aggregator[] concurrentGet(int offset) + @Nullable + @Override + public QueryableProjection getProjection(CursorBuildSpec buildSpec) { - // All get operations should be fine - return aggregators.get(offset); + return Projections.findMatchingProjection( + buildSpec, + aggregateProjections, + (specName, columnName) -> projections.get(specName).getDimensionsMap().containsKey(columnName) + || getColumnCapabilities(columnName) == null, + projections::get + ); } - protected void concurrentSet(int offset, Aggregator[] value) + @Override + public IncrementalIndexRowSelector getProjection(String name) { - aggregators.put(offset, value); + return projections.get(name); } @Override public boolean canAppendRow() { - final boolean countCheck = size() < maxRowCount; + final boolean countCheck = numRows() < maxRowCount; // if maxBytesInMemory = -1, then ignore sizeCheck final boolean sizeCheck = maxBytesInMemory <= 0 || getBytesInMemory().get() < maxBytesInMemory; final boolean canAdd = countCheck && sizeCheck; @@ -459,42 +590,53 @@ public String getOutOfRowsReason() return outOfRowsReason; } - protected Aggregator[] getAggsForRow(int rowOffset) - { - return concurrentGet(rowOffset); - } - @Override public float getMetricFloatValue(int rowOffset, int aggOffset) { - return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getFloat)).floatValue(); + return ((Number) getMetricHelper( + getMetricAggs(), + aggregators.get(rowOffset), + aggOffset, + Aggregator::getFloat + )).floatValue(); } @Override public long getMetricLongValue(int rowOffset, int aggOffset) { - return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getLong)).longValue(); + return ((Number) getMetricHelper( + getMetricAggs(), + aggregators.get(rowOffset), + aggOffset, + Aggregator::getLong + )).longValue(); } @Override - public Object getMetricObjectValue(int rowOffset, int aggOffset) + public double getMetricDoubleValue(int rowOffset, int aggOffset) { - return getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::get); + return ((Number) getMetricHelper( + getMetricAggs(), + aggregators.get(rowOffset), + aggOffset, + Aggregator::getDouble + )).doubleValue(); } @Override - protected double getMetricDoubleValue(int rowOffset, int aggOffset) + public Object getMetricObjectValue(int rowOffset, int aggOffset) { - return ((Number) getMetricHelper(getMetricAggs(), concurrentGet(rowOffset), aggOffset, Aggregator::getDouble)).doubleValue(); + return getMetricHelper(getMetricAggs(), aggregators.get(rowOffset), aggOffset, Aggregator::get); } @Override public boolean isNull(int rowOffset, int aggOffset) { + final Aggregator[] aggs = aggregators.get(rowOffset); if (preserveExistingMetrics) { - return concurrentGet(rowOffset)[aggOffset].isNull() && concurrentGet(rowOffset)[aggOffset + getMetricAggs().length].isNull(); + return aggs[aggOffset].isNull() && aggs[aggOffset + getMetricAggs().length].isNull(); } else { - return concurrentGet(rowOffset)[aggOffset].isNull(); + return aggs[aggOffset].isNull(); } } @@ -535,7 +677,7 @@ public Iterable iterableWithPostAggregations( theVals.put(dimensionName, rowVals); } - Aggregator[] aggs = getAggsForRow(rowOffset); + Aggregator[] aggs = aggregators.get(rowOffset); int aggLength = preserveExistingMetrics ? aggs.length / 2 : aggs.length; for (int i = 0; i < aggLength; ++i) { theVals.put(metrics[i].getName(), getMetricHelper(metrics, aggs, i, Aggregator::get)); @@ -560,11 +702,16 @@ public Iterable iterableWithPostAggregations( * for aggregating from input into output field and the aggregator for combining already aggregated field, as needed */ @Nullable - private Object getMetricHelper(AggregatorFactory[] metrics, Aggregator[] aggs, int aggOffset, Function getMetricTypeFunction) + private Object getMetricHelper( + AggregatorFactory[] metrics, + Aggregator[] aggs, + int aggOffset, + Function getMetricTypeFunction + ) { if (preserveExistingMetrics) { - // Since the preserveExistingMetrics flag is set, we will have to check and possibly retrieve the aggregated values - // from two aggregators, the aggregator for aggregating from input into output field and the aggregator + // Since the preserveExistingMetrics flag is set, we will have to check and possibly retrieve the aggregated + // values from two aggregators, the aggregator for aggregating from input into output field and the aggregator // for combining already aggregated field if (aggs[aggOffset].isNull()) { // If the aggregator for aggregating from input into output field is null, then we get the value from the @@ -583,8 +730,8 @@ private Object getMetricHelper(AggregatorFactory[] metrics, Aggregator[] agg return aggregatorFactory.combine(aggregatedFromSource, aggregatedFromCombined); } } else { - // If preserveExistingMetrics flag is not set then we simply get metrics from the list of Aggregator, aggs, using the - // given aggOffset + // If preserveExistingMetrics flag is not set then we simply get metrics from the list of Aggregator, aggs, + // using the given aggOffset return getMetricTypeFunction.apply(aggs[aggOffset]); } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java index bfccf74c17be..4569df39ef66 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java @@ -100,6 +100,12 @@ public DictionaryIdLookup( this.arrayDictionaryWriter = arrayDictionaryWriter; } + public int[] getArrayValue(int id) + { + ensureArrayDictionaryLoaded(); + return arrayDictionary.get(id - arrayOffset()); + } + @Nullable public Object getDictionaryValue(int id) { @@ -401,4 +407,28 @@ public int addToOffset(long numBytesWritten) } }; } + + public int getStringCardinality() + { + ensureStringDictionaryLoaded(); + return stringDictionary == null ? 0 : stringDictionary.size(); + } + + public int getLongCardinality() + { + ensureLongDictionaryLoaded(); + return longDictionary == null ? 0 : longDictionary.size(); + } + + public int getDoubleCardinality() + { + ensureDoubleDictionaryLoaded(); + return doubleDictionary == null ? 0 : doubleDictionary.size(); + } + + public int getArrayCardinality() + { + ensureArrayDictionaryLoaded(); + return arrayDictionary == null ? 0 : arrayDictionary.size(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java index e630b64d7d62..50bca9977355 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumnSerializer.java @@ -69,7 +69,9 @@ public abstract void serializeDictionaries( public abstract String getColumnName(); - public abstract DictionaryIdLookup getGlobalLookup(); + public abstract DictionaryIdLookup getDictionaryIdLookup(); + + public abstract void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup); public abstract boolean hasNulls(); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java index 7dea3b664fd4..6a3405e58fc2 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSerializer.java @@ -156,6 +156,7 @@ public ProcessedValue processArrayField( private Map> fieldWriters; private int rowCount = 0; private boolean closedForWrite = false; + private boolean writeDictionary = true; private boolean dictionarySerialized = false; private ByteBuffer columnNameBytes = null; @@ -180,11 +181,19 @@ public String getColumnName() } @Override - public DictionaryIdLookup getGlobalLookup() + public DictionaryIdLookup getDictionaryIdLookup() { return globalDictionaryIdLookup; } + @Override + public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup) + { + this.globalDictionaryIdLookup = dictionaryIdLookup; + this.writeDictionary = false; + this.dictionarySerialized = true; + } + @Override public boolean hasNulls() { @@ -421,35 +430,50 @@ public void writeTo( ) throws IOException { Preconditions.checkState(closedForWrite, "Not closed yet!"); - Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + if (writeDictionary) { + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + } writeV0Header(channel, columnNameBytes); fieldsWriter.writeTo(channel, smoosher); fieldsInfoWriter.writeTo(channel, smoosher); - - if (globalDictionaryIdLookup.getStringBufferMapper() != null) { - SmooshedFileMapper fileMapper = globalDictionaryIdLookup.getStringBufferMapper(); - for (String internalName : fileMapper.getInternalFilenames()) { - smoosher.add(internalName, fileMapper.mapFile(internalName)); + if (writeDictionary) { + if (globalDictionaryIdLookup.getStringBufferMapper() != null) { + SmooshedFileMapper fileMapper = globalDictionaryIdLookup.getStringBufferMapper(); + for (String internalName : fileMapper.getInternalFilenames()) { + smoosher.add(internalName, fileMapper.mapFile(internalName)); + } + } else { + writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); + } + if (globalDictionaryIdLookup.getLongBuffer() != null) { + writeInternal( + smoosher, + globalDictionaryIdLookup.getLongBuffer(), + ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); + } + if (globalDictionaryIdLookup.getDoubleBuffer() != null) { + writeInternal( + smoosher, + globalDictionaryIdLookup.getDoubleBuffer(), + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); + } + if (globalDictionaryIdLookup.getArrayBuffer() != null) { + writeInternal( + smoosher, + globalDictionaryIdLookup.getArrayBuffer(), + ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); } - } else { - writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); - } - if (globalDictionaryIdLookup.getLongBuffer() != null) { - writeInternal(smoosher, globalDictionaryIdLookup.getLongBuffer(), ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } - if (globalDictionaryIdLookup.getDoubleBuffer() != null) { - writeInternal(smoosher, globalDictionaryIdLookup.getDoubleBuffer(), ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } - if (globalDictionaryIdLookup.getArrayBuffer() != null) { - writeInternal(smoosher, globalDictionaryIdLookup.getArrayBuffer(), ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); } writeInternal(smoosher, rawWriter, RAW_FILE_NAME); if (!nullRowsBitmap.isEmpty()) { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java index 7c5e722ca67a..7b5678be0744 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataColumnSupplier.java @@ -55,7 +55,8 @@ public static NestedDataColumnSupplier read( ColumnBuilder columnBuilder, ColumnConfig columnConfig, BitmapSerdeFactory bitmapSerdeFactory, - ByteOrder byteOrder + ByteOrder byteOrder, + NestedDataColumnSupplier parent ) { final byte version = bb.get(); @@ -77,49 +78,57 @@ public static NestedDataColumnSupplier read( fields = GenericIndexed.read(bb, GenericIndexed.STRING_STRATEGY, mapper); fieldInfo = FieldTypeInfo.read(bb, fields.size()); - final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME - ); + if (parent != null) { + stringDictionarySupplier = parent.stringDictionarySupplier; + longDictionarySupplier = parent.longDictionarySupplier; + doubleDictionarySupplier = parent.doubleDictionarySupplier; + arrayDictionarySupplier = parent.arrayDictionarySupplier; + } else { + final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME + ); + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME + ); + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME + ); + + stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( + mapper, + stringDictionaryBuffer, + byteOrder + ); + longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + ColumnType.LONG.getStrategy(), + byteOrder, + Long.BYTES + ); + doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + ColumnType.DOUBLE.getStrategy(), + byteOrder, + Double.BYTES + ); + arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( + arrayDictionarybuffer, + byteOrder + ); + } - stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( - mapper, - stringDictionaryBuffer, - byteOrder - ); - final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME - ); - longDictionarySupplier = FixedIndexed.read( - longDictionaryBuffer, - ColumnType.LONG.getStrategy(), - byteOrder, - Long.BYTES - ); - final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME - ); - doubleDictionarySupplier = FixedIndexed.read( - doubleDictionaryBuffer, - ColumnType.DOUBLE.getStrategy(), - byteOrder, - Double.BYTES - ); - final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME - ); - arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( - arrayDictionarybuffer, - byteOrder - ); final ByteBuffer rawBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java index 5199c98d93f0..6fa2fe6d0f08 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java @@ -91,7 +91,8 @@ public static ScalarDoubleColumnAndIndexSupplier read( BitmapSerdeFactory bitmapSerdeFactory, ByteBuffer bb, ColumnBuilder columnBuilder, - ColumnConfig columnConfig + ColumnConfig columnConfig, + @Nullable ScalarDoubleColumnAndIndexSupplier parent ) { final byte version = bb.get(); @@ -103,23 +104,29 @@ public static ScalarDoubleColumnAndIndexSupplier read( final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); - final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME - ); final ByteBuffer doublesValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, ColumnSerializerUtils.DOUBLE_VALUE_COLUMN_FILE_NAME ); - final Supplier> doubleDictionarySupplier = FixedIndexed.read( - doubleDictionaryBuffer, - ColumnType.DOUBLE.getStrategy(), - byteOrder, - Double.BYTES - ); + final Supplier> doubleDictionarySupplier; + if (parent != null) { + doubleDictionarySupplier = parent.doubleDictionarySupplier; + } else { + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + + doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + ColumnType.DOUBLE.getStrategy(), + byteOrder, + Double.BYTES + ); + } final Supplier doubles = CompressedColumnarDoublesSuppliers.fromByteBuffer( doublesValueColumn, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java index 718981ae19a4..050848f65036 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnSerializer.java @@ -142,4 +142,13 @@ protected void writeDictionaryFile(FileSmoosher smoosher) throws IOException writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); } } + + @Override + public int getCardinality() + { + if (writeDictionary) { + return dictionaryWriter.getCardinality(); + } + return dictionaryIdLookup.getDoubleCardinality(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java index 66527530336c..a8d1fa057d86 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java @@ -90,7 +90,8 @@ public static ScalarLongColumnAndIndexSupplier read( BitmapSerdeFactory bitmapSerdeFactory, ByteBuffer bb, ColumnBuilder columnBuilder, - ColumnConfig columnConfig + ColumnConfig columnConfig, + @Nullable ScalarLongColumnAndIndexSupplier parent ) { final byte version = bb.get(); @@ -102,11 +103,6 @@ public static ScalarLongColumnAndIndexSupplier read( final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); - final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME - ); final ByteBuffer longsValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, @@ -123,12 +119,23 @@ public static ScalarLongColumnAndIndexSupplier read( columnBuilder.getFileMapper() ); - final Supplier> longDictionarySupplier = FixedIndexed.read( - longDictionaryBuffer, - ColumnType.LONG.getStrategy(), - byteOrder, - Long.BYTES - ); + + final Supplier> longDictionarySupplier; + if (parent != null) { + longDictionarySupplier = parent.longDictionarySupplier; + } else { + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME + ); + longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + ColumnType.LONG.getStrategy(), + byteOrder, + Long.BYTES + ); + } final Supplier longs = CompressedColumnarLongsSupplier.fromByteBuffer( longsValueColumn, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java index 00d7c16813d6..89408b203884 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnSerializer.java @@ -142,4 +142,13 @@ protected void writeDictionaryFile(FileSmoosher smoosher) throws IOException writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); } } + + @Override + public int getCardinality() + { + if (writeDictionary) { + return dictionaryWriter.getCardinality(); + } + return dictionaryIdLookup.getLongCardinality(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java index 1010003a6115..dc18970ad799 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarNestedCommonFormatColumnSerializer.java @@ -62,6 +62,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer extends Nested protected ByteBuffer columnNameBytes = null; protected boolean hasNulls; + protected boolean writeDictionary = true; public ScalarNestedCommonFormatColumnSerializer( @@ -98,6 +99,8 @@ public ScalarNestedCommonFormatColumnSerializer( protected abstract void writeDictionaryFile(FileSmoosher smoosher) throws IOException; + public abstract int getCardinality(); + @Override public String getColumnName() { @@ -105,11 +108,19 @@ public String getColumnName() } @Override - public DictionaryIdLookup getGlobalLookup() + public DictionaryIdLookup getDictionaryIdLookup() { return dictionaryIdLookup; } + @Override + public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup) + { + this.dictionaryIdLookup = dictionaryIdLookup; + this.writeDictionary = false; + this.dictionarySerialized = true; + } + @Override public boolean hasNulls() { @@ -166,7 +177,9 @@ public void writeTo( ) throws IOException { Preconditions.checkState(closedForWrite, "Not closed yet!"); - Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + if (writeDictionary) { + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + } // write out compressed dictionaryId int column and bitmap indexes by iterating intermediate value column // the intermediate value column should be replaced someday by a cooler compressed int column writer that allows @@ -184,7 +197,7 @@ public void writeTo( name, segmentWriteOutMedium, filenameBase, - dictionaryWriter.getCardinality(), + getCardinality(), compressionToUse, segmentWriteOutMedium.getCloser() ); @@ -198,7 +211,7 @@ public void writeTo( bitmapIndexWriter.open(); bitmapIndexWriter.setObjectsNotSorted(); final MutableBitmap[] bitmaps; - bitmaps = new MutableBitmap[dictionaryWriter.getCardinality()]; + bitmaps = new MutableBitmap[getCardinality()]; for (int i = 0; i < bitmaps.length; i++) { bitmaps[i] = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } @@ -220,7 +233,9 @@ public void writeTo( } writeV0Header(channel, columnNameBytes); - writeDictionaryFile(smoosher); + if (writeDictionary) { + writeDictionaryFile(smoosher); + } writeInternal(smoosher, encodedValueSerializer, ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME); writeValueColumn(smoosher); writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java index c4850905b7fa..7f1111c2e8b5 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java @@ -49,7 +49,8 @@ public static ScalarStringColumnAndIndexSupplier read( ByteOrder byteOrder, BitmapSerdeFactory bitmapSerdeFactory, ByteBuffer bb, - ColumnBuilder columnBuilder + ColumnBuilder columnBuilder, + @Nullable ScalarStringColumnAndIndexSupplier parent ) { final byte version = bb.get(); @@ -61,17 +62,22 @@ public static ScalarStringColumnAndIndexSupplier read( final SmooshedFileMapper mapper = columnBuilder.getFileMapper(); final Supplier> dictionarySupplier; - final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME - ); + if (parent != null) { + dictionarySupplier = parent.dictionarySupplier; + } else { + final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME + ); + + dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( + mapper, + stringDictionaryBuffer, + byteOrder + ); + } - dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( - mapper, - stringDictionaryBuffer, - byteOrder - ); final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, @@ -106,6 +112,7 @@ public static ScalarStringColumnAndIndexSupplier read( } } + private final Supplier> dictionarySupplier; private final Supplier encodedColumnSupplier; private final GenericIndexed valueIndexes; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java index 380da1ea9827..230a9433cdcf 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnSerializer.java @@ -131,4 +131,13 @@ protected void writeDictionaryFile(FileSmoosher smoosher) throws IOException writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); } } + + @Override + public int getCardinality() + { + if (writeDictionary) { + return dictionaryWriter.getCardinality(); + } + return dictionaryIdLookup.getStringCardinality(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java index 3cd9e4f7308a..23555b2ea2d6 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java @@ -68,7 +68,8 @@ public static VariantColumnAndIndexSupplier read( ByteOrder byteOrder, BitmapSerdeFactory bitmapSerdeFactory, ByteBuffer bb, - ColumnBuilder columnBuilder + ColumnBuilder columnBuilder, + @Nullable VariantColumnAndIndexSupplier parent ) { final byte version = bb.get(); @@ -95,17 +96,68 @@ public static VariantColumnAndIndexSupplier read( final Supplier arrayDictionarySupplier; final Supplier> arrayElementDictionarySupplier; - final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME - ); + if (parent != null) { + stringDictionarySupplier = parent.stringDictionarySupplier; + longDictionarySupplier = parent.longDictionarySupplier; + doubleDictionarySupplier = parent.doubleDictionarySupplier; + arrayDictionarySupplier = parent.arrayDictionarySupplier; + arrayElementDictionarySupplier = parent.arrayElementDictionarySupplier; + } else { + final ByteBuffer stringDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME + ); + final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME + ); + final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME + ); + final ByteBuffer arrayElementDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME + ); + + stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( + mapper, + stringDictionaryBuffer, + byteOrder + ); + longDictionarySupplier = FixedIndexed.read( + longDictionaryBuffer, + ColumnType.LONG.getStrategy(), + byteOrder, + Long.BYTES + ); + doubleDictionarySupplier = FixedIndexed.read( + doubleDictionaryBuffer, + ColumnType.DOUBLE.getStrategy(), + byteOrder, + Double.BYTES + ); + arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( + arrayDictionarybuffer, + byteOrder + ); + arrayElementDictionarySupplier = FixedIndexed.read( + arrayElementDictionaryBuffer, + CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, + byteOrder, + Integer.BYTES + ); + } - stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( - mapper, - stringDictionaryBuffer, - byteOrder - ); final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, @@ -115,21 +167,6 @@ public static VariantColumnAndIndexSupplier read( encodedValueColumn, byteOrder ); - final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME - ); - final ByteBuffer doubleDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME - ); - final ByteBuffer arrayElementDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME - ); final ByteBuffer valueIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( mapper, columnName, @@ -151,34 +188,6 @@ public static VariantColumnAndIndexSupplier read( columnBuilder.getFileMapper() ); - longDictionarySupplier = FixedIndexed.read( - longDictionaryBuffer, - ColumnType.LONG.getStrategy(), - byteOrder, - Long.BYTES - ); - doubleDictionarySupplier = FixedIndexed.read( - doubleDictionaryBuffer, - ColumnType.DOUBLE.getStrategy(), - byteOrder, - Double.BYTES - ); - - final ByteBuffer arrayDictionarybuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( - mapper, - columnName, - ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME - ); - arrayDictionarySupplier = FrontCodedIntArrayIndexed.read( - arrayDictionarybuffer, - byteOrder - ); - arrayElementDictionarySupplier = FixedIndexed.read( - arrayElementDictionaryBuffer, - CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY, - byteOrder, - Integer.BYTES - ); return new VariantColumnAndIndexSupplier( logicalType, variantTypeByte, diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java index b1454d850045..0d07822db541 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnSerializer.java @@ -83,6 +83,7 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer private ByteBuffer columnNameBytes = null; private boolean hasNulls; + private boolean writeDictionary = true; @Nullable private final ExpressionType expectedExpressionType; @Nullable @@ -112,11 +113,19 @@ public String getColumnName() } @Override - public DictionaryIdLookup getGlobalLookup() + public DictionaryIdLookup getDictionaryIdLookup() { return dictionaryIdLookup; } + @Override + public void setDictionaryIdLookup(DictionaryIdLookup dictionaryIdLookup) + { + this.dictionaryIdLookup = dictionaryIdLookup; + this.writeDictionary = false; + this.dictionarySerialized = true; + } + @Override public boolean hasNulls() { @@ -319,17 +328,19 @@ public void writeTo( ) throws IOException { Preconditions.checkState(closedForWrite, "Not closed yet!"); - Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + if (writeDictionary) { + Preconditions.checkArgument(dictionaryWriter.isSorted(), "Dictionary not sorted?!?"); + } // write out compressed dictionaryId int column, bitmap indexes, and array element bitmap indexes // by iterating intermediate value column the intermediate value column should be replaced someday by a cooler // compressed int column writer that allows easy iteration of the values it writes out, so that we could just // build the bitmap indexes here instead of doing both things String filenameBase = StringUtils.format("%s.forward_dim", name); - final int cardinality = dictionaryWriter.getCardinality() - + longDictionaryWriter.getCardinality() - + doubleDictionaryWriter.getCardinality() - + arrayDictionaryWriter.getCardinality(); + final int cardinality = dictionaryIdLookup.getStringCardinality() + + dictionaryIdLookup.getLongCardinality() + + dictionaryIdLookup.getDoubleCardinality() + + dictionaryIdLookup.getArrayCardinality(); final CompressionStrategy compression = indexSpec.getDimensionCompression(); final CompressionStrategy compressionToUse; if (compression != CompressionStrategy.UNCOMPRESSED && compression != CompressionStrategy.NONE) { @@ -370,15 +381,15 @@ public void writeTo( final IntIterator rows = intermediateValueWriter.getIterator(); int rowCount = 0; - final int arrayBaseId = dictionaryWriter.getCardinality() - + longDictionaryWriter.getCardinality() - + doubleDictionaryWriter.getCardinality(); + final int arrayBaseId = dictionaryIdLookup.getStringCardinality() + + dictionaryIdLookup.getLongCardinality() + + dictionaryIdLookup.getDoubleCardinality(); while (rows.hasNext()) { final int dictId = rows.nextInt(); encodedValueSerializer.addValue(dictId); bitmaps[dictId].add(rowCount); if (dictId >= arrayBaseId) { - int[] array = arrayDictionaryWriter.get(dictId - arrayBaseId); + int[] array = dictionaryIdLookup.getArrayValue(dictId); for (int elementId : array) { arrayElements.computeIfAbsent( elementId, @@ -396,11 +407,13 @@ public void writeTo( ); bitmaps[i] = null; // Reclaim memory } - for (Int2ObjectMap.Entry arrayElement : arrayElements.int2ObjectEntrySet()) { - arrayElementDictionaryWriter.write(arrayElement.getIntKey()); - arrayElementIndexWriter.write( - indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) - ); + if (writeDictionary) { + for (Int2ObjectMap.Entry arrayElement : arrayElements.int2ObjectEntrySet()) { + arrayElementDictionaryWriter.write(arrayElement.getIntKey()); + arrayElementIndexWriter.write( + indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeImmutableBitmap(arrayElement.getValue()) + ); + } } writeV0Header(channel, columnNameBytes); @@ -408,31 +421,37 @@ public void writeTo( channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte})); } - if (dictionaryIdLookup.getStringBufferMapper() != null) { - SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper(); - for (String internalName : fileMapper.getInternalFilenames()) { - smoosher.add(internalName, fileMapper.mapFile(internalName)); + if (writeDictionary) { + if (dictionaryIdLookup.getStringBufferMapper() != null) { + SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper(); + for (String internalName : fileMapper.getInternalFilenames()) { + smoosher.add(internalName, fileMapper.mapFile(internalName)); + } + } else { + writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); + } + if (dictionaryIdLookup.getLongBuffer() != null) { + writeInternal(smoosher, dictionaryIdLookup.getLongBuffer(), ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); + } else { + writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); + } + if (dictionaryIdLookup.getDoubleBuffer() != null) { + writeInternal( + smoosher, + dictionaryIdLookup.getDoubleBuffer(), + ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME + ); + } else { + writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); + } + if (dictionaryIdLookup.getArrayBuffer() != null) { + writeInternal(smoosher, dictionaryIdLookup.getArrayBuffer(), ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); + } else { + writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); } - } else { - writeInternal(smoosher, dictionaryWriter, ColumnSerializerUtils.STRING_DICTIONARY_FILE_NAME); - } - if (dictionaryIdLookup.getLongBuffer() != null) { - writeInternal(smoosher, dictionaryIdLookup.getLongBuffer(), ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, longDictionaryWriter, ColumnSerializerUtils.LONG_DICTIONARY_FILE_NAME); - } - if (dictionaryIdLookup.getDoubleBuffer() != null) { - writeInternal(smoosher, dictionaryIdLookup.getDoubleBuffer(), ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, doubleDictionaryWriter, ColumnSerializerUtils.DOUBLE_DICTIONARY_FILE_NAME); - } - if (dictionaryIdLookup.getArrayBuffer() != null) { - writeInternal(smoosher, dictionaryIdLookup.getArrayBuffer(), ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); - } else { - writeInternal(smoosher, arrayDictionaryWriter, ColumnSerializerUtils.ARRAY_DICTIONARY_FILE_NAME); - } - writeInternal(smoosher, arrayElementDictionaryWriter, ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME); + writeInternal(smoosher, arrayElementDictionaryWriter, ColumnSerializerUtils.ARRAY_ELEMENT_DICTIONARY_FILE_NAME); + } writeInternal(smoosher, encodedValueSerializer, ColumnSerializerUtils.ENCODED_VALUE_COLUMN_FILE_NAME); writeInternal(smoosher, bitmapIndexWriter, ColumnSerializerUtils.BITMAP_INDEX_FILE_NAME); writeInternal(smoosher, arrayElementIndexWriter, ColumnSerializerUtils.ARRAY_ELEMENT_BITMAP_INDEX_FILE_NAME); diff --git a/processing/src/main/java/org/apache/druid/segment/projections/Projections.java b/processing/src/main/java/org/apache/druid/segment/projections/Projections.java new file mode 100644 index 000000000000..c38a875226f7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/projections/Projections.java @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.projections; + +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.AggregateProjectionMetadata; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.ConstantExprEvalSelector; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.CapabilitiesBasedFormat; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.NumericColumn; +import org.apache.druid.segment.data.ReadableOffset; +import org.apache.druid.segment.vector.ConstantVectorSelectors; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.VectorValueSelector; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.function.Function; + +public class Projections +{ + public static Supplier makeConstantTimeSupplier(int numRows, long constant) + { + return Suppliers.memoize( + () -> new ColumnBuilder().setNumericColumnSupplier(() -> new ConstantTimeColumn(numRows, constant)) + .setColumnFormat( + new CapabilitiesBasedFormat( + ColumnCapabilitiesImpl.createDefault().setType(ColumnType.LONG) + ) + ) + .setType(ColumnType.LONG) + .setHasNulls(false) + .build() + ); + } + + @Nullable + public static QueryableProjection findMatchingProjection( + CursorBuildSpec cursorBuildSpec, + SortedSet projections, + PhysicalColumnChecker physicalChecker, + Function getRowSelector + ) + { + if (cursorBuildSpec.getQueryContext().getBoolean(QueryContexts.NO_PROJECTIONS, false)) { + return null; + } + final String name = cursorBuildSpec.getQueryContext().getString(QueryContexts.USE_PROJECTION); + + if (cursorBuildSpec.isAggregate()) { + for (AggregateProjectionMetadata spec : projections) { + if (name != null && !name.equals(spec.getSchema().getName())) { + continue; + } + ProjectionMatch match = spec.getSchema().matches(cursorBuildSpec, physicalChecker); + if (match != null) { + if (cursorBuildSpec.getQueryMetrics() != null) { + cursorBuildSpec.getQueryMetrics().projection(spec.getSchema().getName()); + } + return new QueryableProjection<>( + match.getCursorBuildSpec(), + match.getRemapColumns(), + getRowSelector.apply(spec.getSchema().getName()) + ); + } + } + } + if (name != null) { + throw InvalidInput.exception("Projection[%s] specified, but does not satisfy query", name); + } + if (cursorBuildSpec.getQueryContext().getBoolean(QueryContexts.FORCE_PROJECTION, false)) { + throw InvalidInput.exception("Force projections specified, but none satisfy query"); + } + return null; + } + + public static String getProjectionSmooshV9FileName(AggregateProjectionMetadata projectionSpec, String columnName) + { + return getProjectionSmooshV9Prefix(projectionSpec) + columnName; + } + + public static String getProjectionSmooshV9Prefix(AggregateProjectionMetadata projectionSpec) + { + return projectionSpec.getSchema().getName() + "/"; + } + + /** + * Returns true if column is defined in {@link AggregateProjectionSpec#getGroupingColumns()} OR if the column does not + * exist in the base table. Part of determining if a projection can be used for a given {@link CursorBuildSpec}, + * + * @see AggregateProjectionMetadata.Schema#matches(CursorBuildSpec, PhysicalColumnChecker) + */ + @FunctionalInterface + public interface PhysicalColumnChecker + { + boolean check(String projectionName, String columnName); + } + + public static final class ProjectionMatch + { + private final CursorBuildSpec cursorBuildSpec; + private final Map remapColumns; + + public ProjectionMatch(CursorBuildSpec cursorBuildSpec, Map remapColumns) + { + this.cursorBuildSpec = cursorBuildSpec; + this.remapColumns = remapColumns; + } + + public CursorBuildSpec getCursorBuildSpec() + { + return cursorBuildSpec; + } + + public Map getRemapColumns() + { + return remapColumns; + } + } + + public static final class ProjectionMatchBuilder + { + private final Set referencedVirtualColumns; + private final Map remapColumns; + private final List combiningFactories; + + public ProjectionMatchBuilder() + { + this.referencedVirtualColumns = new HashSet<>(); + this.remapColumns = new HashMap<>(); + this.combiningFactories = new ArrayList<>(); + } + + /** + * Map a query column name to a projection column name + */ + public ProjectionMatchBuilder remapColumn(String queryColumn, String projectionColumn) + { + remapColumns.put(queryColumn, projectionColumn); + return this; + } + + /** + * Add a query virtual column that can use projection physical columns as inputs to the match builder, which will + * later be added to {@link ProjectionMatch#getCursorBuildSpec()} if the projection matches + */ + public ProjectionMatchBuilder addReferenceedVirtualColumn(VirtualColumn virtualColumn) + { + referencedVirtualColumns.add(virtualColumn); + return this; + } + + /** + * Add a query {@link AggregatorFactory#substituteCombiningFactory(AggregatorFactory)} which can combine the inputs + * of a selector created by a projection {@link AggregatorFactory} + * + */ + public ProjectionMatchBuilder addPreAggregatedAggregator(AggregatorFactory aggregator) + { + combiningFactories.add(aggregator); + return this; + } + + public ProjectionMatch build(CursorBuildSpec queryCursorBuildSpec) + { + return new ProjectionMatch( + CursorBuildSpec.builder(queryCursorBuildSpec) + .setVirtualColumns(VirtualColumns.fromIterable(referencedVirtualColumns)) + .setAggregators(combiningFactories) + .build(), + remapColumns + ); + } + } + + private static class ConstantTimeColumn implements NumericColumn + { + private final int numRows; + private final long constant; + + private ConstantTimeColumn(int numRows, long constant) + { + this.numRows = numRows; + this.constant = constant; + } + + @Override + public int length() + { + return numRows; + } + + @Override + public long getLongSingleValueRow(int rowNum) + { + return constant; + } + + @Override + public void close() + { + // nothing to close + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + + @Override + public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) + { + return new ConstantExprEvalSelector(ExprEval.ofLong(constant)); + } + + @Override + public VectorValueSelector makeVectorValueSelector(ReadableVectorOffset offset) + { + return ConstantVectorSelectors.vectorValueSelector(offset, constant); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/projections/QueryableProjection.java b/processing/src/main/java/org/apache/druid/segment/projections/QueryableProjection.java new file mode 100644 index 000000000000..b32ea829f34d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/projections/QueryableProjection.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.projections; + +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.CursorBuildSpec; +import org.apache.druid.segment.RemapColumnSelectorFactory; +import org.apache.druid.segment.vector.RemapVectorColumnSelectorFactory; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import java.util.Map; + +/** + * Represents a projection of some base table available to use to build a {@link org.apache.druid.segment.CursorHolder} + * by a {@link org.apache.druid.segment.CursorFactory}. + *

+ * Projections are a type of invisible materialized view stored inside of a {@link org.apache.druid.segment.Segment} + * which can be automatically used if they match the {@link CursorBuildSpec} argument passed to + * {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)}. + *

+ * In the most basic sense, a projection consists of: + * - the actual underlying projection rows ({@link #rowSelector}) + * - a mapping of {@link CursorBuildSpec} columns to underlying projection columns ({@link #remapColumns}) + * - and a modified {@link CursorBuildSpec} ({@link #cursorBuildSpec}) + *

+ * The {@link #getRowSelector()} and {@link #getCursorBuildSpec()} methods can be used by a + * {@link org.apache.druid.segment.CursorFactory} to build a {@link org.apache.druid.segment.CursorHolder} for the + * projection instead of the base table, and {@link #wrapColumnSelectorFactory(ColumnSelectorFactory)} and + * {@link #wrapVectorColumnSelectorFactory(VectorColumnSelectorFactory)} can be used to decorate the selector factories + * constructed by that {@link org.apache.druid.segment.CursorHolder} whenever it builds a + * {@link org.apache.druid.segment.Cursor} or {@link org.apache.druid.segment.vector.VectorCursor} to ensure that all + * the selectors needed to satisfy the original {@link CursorBuildSpec} are available at the correct names. + * + * @see org.apache.druid.segment.QueryableIndexCursorFactory#makeCursorHolder(CursorBuildSpec) + * @see org.apache.druid.segment.incremental.IncrementalIndexCursorFactory#makeCursorHolder(CursorBuildSpec) + */ +public class QueryableProjection +{ + private final CursorBuildSpec cursorBuildSpec; + private final Map remapColumns; + private final T rowSelector; + + public QueryableProjection( + CursorBuildSpec cursorBuildSpec, + Map remapColumns, + T rowSelector + ) + { + this.cursorBuildSpec = cursorBuildSpec; + this.remapColumns = remapColumns; + this.rowSelector = rowSelector; + } + + /** + * The original {@link CursorBuildSpec} of a query can be modified if a projection matches the query, such as removing + * virtual columns which have already been pre-computed. + */ + public CursorBuildSpec getCursorBuildSpec() + { + return cursorBuildSpec; + } + + /** + * The projection can contain pre-computed virtual columns or pre-aggregated aggregation columns. At query time, + * these are remapped to match the desired names for all equivalent components of the {@link CursorBuildSpec}. + *

+ * For example, if the original {@link CursorBuildSpec} has a sum aggregator named 'sum_x' which takes a field 'x' + * as input, and an equivalent sum aggregation exists on the projection with the name 'xsum' built from the base table + * column 'x', the wrapped column selector factory will make 'xsum' available as 'sum_x', allowing the query to + * use the combining aggregator instead of processing the base table for column 'x'. + */ + public ColumnSelectorFactory wrapColumnSelectorFactory(ColumnSelectorFactory selectorFactory) + { + return new RemapColumnSelectorFactory( + selectorFactory, + remapColumns + ); + } + + /** + * The projection can contain pre-computed virtual columns or pre-aggregated aggregation columns. At query time, + * these are remapped to match the desired names for all equivalent components of the {@link CursorBuildSpec} + *

+ * For example, if the original {@link CursorBuildSpec} has a sum aggregator named 'sum_x' which takes a field 'x' + * as input, and an equivalent sum aggregation exists on the projection with the name 'xsum' built from the base table + * column 'x', the wrapped column selector factory will make 'xsum' available as 'sum_x', allowing the query to + * use the combining aggregator instead of processing the base table for column 'x'. + */ + public VectorColumnSelectorFactory wrapVectorColumnSelectorFactory(VectorColumnSelectorFactory selectorFactory) + { + return new RemapVectorColumnSelectorFactory(selectorFactory, remapColumns); + } + + /** + * Backing storage for the rows of the projection as is appropriate for the type of + * {@link org.apache.druid.segment.CursorFactory} + */ + public T getRowSelector() + { + return rowSelector; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java index 19008a36526b..f48b46fbbdde 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/ColumnPartSerde.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -54,6 +55,11 @@ public interface ColumnPartSerde interface Deserializer { - void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig); + void read( + ByteBuffer buffer, + ColumnBuilder builder, + ColumnConfig columnConfig, + @Nullable ColumnHolder parent + ); } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java index 651d2c6a5d9a..f8220d40046e 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/ComplexColumnPartSerde.java @@ -77,7 +77,7 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { // we don't currently know if complex column can have nulls (or can be multi-valued, but not making that change // since it isn't supported anywhere in the query engines) // longer term this needs to be captured by making the serde provide this information, and then this should diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index 41aef13f576c..cf9b7c70d563 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -29,8 +29,10 @@ import org.apache.druid.io.Channels; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerde; @@ -292,7 +294,12 @@ public Deserializer getDeserializer() return new Deserializer() { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read( + ByteBuffer buffer, + ColumnBuilder builder, + ColumnConfig columnConfig, + @Nullable ColumnHolder parent + ) { final VERSION rVersion = VERSION.fromByte(buffer.get()); final int rFlags; @@ -309,12 +316,17 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setType(ValueType.STRING); - final Supplier> dictionarySupplier = - StringEncodingStrategies.getStringDictionarySupplier( - builder.getFileMapper(), - buffer, - byteOrder - ); + final Supplier> dictionarySupplier; + if (parent != null) { + final Supplier parentSupplier = parent.getColumnSupplier(); + dictionarySupplier = ((StringUtf8DictionaryEncodedColumnSupplier) parentSupplier).getDictionary(); + } else { + dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier( + builder.getFileMapper(), + buffer, + byteOrder + ); + } final WritableSupplier rSingleValuedColumn; final WritableSupplier rMultiValuedColumn; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java index 864e893fd197..012b3ed77b05 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerde.java @@ -96,7 +96,7 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { final Supplier column = CompressedColumnarDoublesSuppliers.fromByteBuffer( buffer, byteOrder diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java index 5eda4f84ae90..0fa42f2187a4 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DoubleNumericColumnPartSerdeV2.java @@ -143,7 +143,7 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { int offset = buffer.getInt(); int initialPos = buffer.position(); final Supplier column = CompressedColumnarDoublesSuppliers.fromByteBuffer( diff --git a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java index 02ac221be32c..441f774c7d17 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerde.java @@ -22,13 +22,10 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.column.ColumnBuilder; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.CompressedColumnarFloatsSupplier; import javax.annotation.Nullable; -import java.nio.ByteBuffer; import java.nio.ByteOrder; /** @@ -99,23 +96,18 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return new Deserializer() - { - @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) - { - final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( - buffer, - byteOrder - ); - FloatNumericColumnSupplier columnSupplier = new FloatNumericColumnSupplier( - column, - IndexIO.LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap() - ); - builder.setType(ValueType.FLOAT) - .setHasMultipleValues(false) - .setNumericColumnSupplier(columnSupplier); - } + return (buffer, builder, columnConfig, parent) -> { + final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( + buffer, + byteOrder + ); + FloatNumericColumnSupplier columnSupplier = new FloatNumericColumnSupplier( + column, + IndexIO.LEGACY_FACTORY.getBitmapFactory().makeEmptyImmutableBitmap() + ); + builder.setType(ValueType.FLOAT) + .setHasMultipleValues(false) + .setNumericColumnSupplier(columnSupplier); }; } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java index 8e79bc7a3fa8..457612c4f817 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/FloatNumericColumnPartSerdeV2.java @@ -141,7 +141,7 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { int offset = buffer.getInt(); int initialPos = buffer.position(); final CompressedColumnarFloatsSupplier column = CompressedColumnarFloatsSupplier.fromByteBuffer( diff --git a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java index 3ad77a45e62d..fa94b91ecd4e 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerde.java @@ -96,7 +96,7 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer( buffer, byteOrder diff --git a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java index 7f145b36a413..ac81a225de4c 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/LongNumericColumnPartSerdeV2.java @@ -143,7 +143,7 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { int offset = buffer.getInt(); int initialPos = buffer.position(); final CompressedColumnarLongsSupplier column = CompressedColumnarLongsSupplier.fromByteBuffer( diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java index e09855cdab2c..d923de51d09d 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -27,6 +27,7 @@ import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; @@ -189,13 +190,14 @@ public BitmapSerdeFactory getBitmapSerdeFactory() private class StringColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { ScalarStringColumnAndIndexSupplier supplier = ScalarStringColumnAndIndexSupplier.read( byteOrder, bitmapSerdeFactory, buffer, - builder + builder, + parent == null ? null : (ScalarStringColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); capabilitiesBuilder.setDictionaryEncoded(true); @@ -205,21 +207,26 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setHasNulls(hasNulls); builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType)); + builder.setColumnFormat(new NestedCommonFormatColumn.Format( + logicalType, + capabilitiesBuilder.hasNulls().isTrue(), + enforceLogicalType + )); } } private class LongColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { ScalarLongColumnAndIndexSupplier supplier = ScalarLongColumnAndIndexSupplier.read( byteOrder, bitmapSerdeFactory, buffer, builder, - columnConfig + columnConfig, + parent == null ? null : (ScalarLongColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); // technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn @@ -228,21 +235,26 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setHasNulls(hasNulls); builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType)); + builder.setColumnFormat(new NestedCommonFormatColumn.Format( + logicalType, + capabilitiesBuilder.hasNulls().isTrue(), + enforceLogicalType + )); } } private class DoubleColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { ScalarDoubleColumnAndIndexSupplier supplier = ScalarDoubleColumnAndIndexSupplier.read( byteOrder, bitmapSerdeFactory, buffer, builder, - columnConfig + columnConfig, + parent == null ? null : (ScalarDoubleColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); // technically, these columns are dictionary encoded, however they do not implement the DictionaryEncodedColumn @@ -251,21 +263,26 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setHasNulls(hasNulls); builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue(), enforceLogicalType)); + builder.setColumnFormat(new NestedCommonFormatColumn.Format( + logicalType, + capabilitiesBuilder.hasNulls().isTrue(), + enforceLogicalType + )); } } private class VariantColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { VariantColumnAndIndexSupplier supplier = VariantColumnAndIndexSupplier.read( logicalType, byteOrder, bitmapSerdeFactory, buffer, - builder + builder, + parent == null ? null : (VariantColumnAndIndexSupplier) parent.getColumnSupplier() ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); // if we are a mixed type, don't call ourself dictionary encoded for now so we don't end up doing the wrong thing @@ -291,7 +308,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo private class NestedColumnDeserializer implements Deserializer { @Override - public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig, @Nullable ColumnHolder parent) { NestedDataColumnSupplier supplier = NestedDataColumnSupplier.read( logicalType, @@ -300,7 +317,8 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder, columnConfig, bitmapSerdeFactory, - byteOrder + byteOrder, + parent == null ? null : (NestedDataColumnSupplier) parent.getColumnSupplier() ); ColumnType simpleType = supplier.getLogicalType(); ColumnType logicalType = simpleType == null ? ColumnType.NESTED_DATA : simpleType; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java index ea8af5e85416..b3489ef01796 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java @@ -105,7 +105,7 @@ public Serializer getSerializer() @Override public Deserializer getDeserializer() { - return (buffer, builder, columnConfig) -> { + return (buffer, builder, columnConfig, parent) -> { builder.setHasMultipleValues(false) .setHasNulls(true) // this is a bit sneaky, we set supplier to null here to act like a null column instead of a column diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java index c0a8a0885e4c..33de1869e273 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8DictionaryEncodedColumnSupplier.java @@ -50,6 +50,11 @@ public StringUtf8DictionaryEncodedColumnSupplier( this.multiValuedColumn = multiValuedColumn; } + public Supplier getDictionary() + { + return utf8Dictionary; + } + @Override public DictionaryEncodedColumn get() { diff --git a/processing/src/main/java/org/apache/druid/segment/vector/RemapVectorColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/vector/RemapVectorColumnSelectorFactory.java new file mode 100644 index 000000000000..0c6bd7b05ecb --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/vector/RemapVectorColumnSelectorFactory.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.vector; + +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.groupby.DeferExpressionDimensions; +import org.apache.druid.query.groupby.epinephelinae.vector.GroupByVectorColumnSelector; +import org.apache.druid.segment.column.ColumnCapabilities; + +import javax.annotation.Nullable; +import java.util.Map; + +public class RemapVectorColumnSelectorFactory implements VectorColumnSelectorFactory +{ + private final VectorColumnSelectorFactory delegate; + private final Map remap; + + public RemapVectorColumnSelectorFactory(VectorColumnSelectorFactory delegate, Map remap) + { + this.delegate = delegate; + this.remap = remap; + } + + @Override + public ReadableVectorInspector getReadableVectorInspector() + { + return delegate.getReadableVectorInspector(); + } + + @Override + public int getMaxVectorSize() + { + return delegate.getMaxVectorSize(); + } + + @Override + public SingleValueDimensionVectorSelector makeSingleValueDimensionSelector(DimensionSpec dimensionSpec) + { + DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension())); + return delegate.makeSingleValueDimensionSelector(remapDimensionSpec); + } + + @Override + public MultiValueDimensionVectorSelector makeMultiValueDimensionSelector(DimensionSpec dimensionSpec) + { + DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension())); + return delegate.makeMultiValueDimensionSelector(remapDimensionSpec); + } + + @Override + public VectorValueSelector makeValueSelector(String column) + { + return delegate.makeValueSelector(remap.getOrDefault(column, column)); + } + + @Override + public VectorObjectSelector makeObjectSelector(String column) + { + return delegate.makeObjectSelector(remap.getOrDefault(column, column)); + } + + @Override + @Nullable + public ColumnCapabilities getColumnCapabilities(String column) + { + return delegate.getColumnCapabilities(remap.getOrDefault(column, column)); + } + + @Nullable + @Override + public ExpressionType getType(String name) + { + return delegate.getType(remap.getOrDefault(name, name)); + } + + @Override + public GroupByVectorColumnSelector makeGroupByVectorColumnSelector( + String column, + DeferExpressionDimensions deferExpressionDimensions + ) + { + return delegate.makeGroupByVectorColumnSelector(remap.getOrDefault(column, column), deferExpressionDimensions); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java index 8bb62128f9da..91c9c9057f4d 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionVirtualColumn.java @@ -63,6 +63,7 @@ public class ExpressionVirtualColumn implements VirtualColumn private final String name; private final Expression expression; private final Supplier parsedExpression; + private final Supplier expressionAnalysis; private final Supplier cacheKey; /** @@ -126,6 +127,7 @@ private ExpressionVirtualColumn( this.name = Preconditions.checkNotNull(name, "name"); this.expression = new Expression(Preconditions.checkNotNull(expression, "expression"), outputType); this.parsedExpression = parsedExpression; + this.expressionAnalysis = Suppliers.memoize(parsedExpression.get()::analyzeInputs); this.cacheKey = makeCacheKeySupplier(); } @@ -332,7 +334,7 @@ public ColumnCapabilities capabilities(ColumnInspector inspector, String columnN @Override public List requiredColumns() { - return parsedExpression.get().analyzeInputs().getRequiredBindingsList(); + return expressionAnalysis.get().getRequiredBindingsList(); } @Override diff --git a/processing/src/test/java/org/apache/druid/data/input/impl/AggregateProjectionSpecTest.java b/processing/src/test/java/org/apache/druid/data/input/impl/AggregateProjectionSpecTest.java new file mode 100644 index 000000000000..60895ceefb01 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/data/input/impl/AggregateProjectionSpecTest.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.data.input.impl; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +public class AggregateProjectionSpecTest extends InitializedNullHandlingTest +{ + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + AggregateProjectionSpec spec = new AggregateProjectionSpec( + "some_projection", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "time") + ), + Arrays.asList( + new StringDimensionSchema("a"), + new LongDimensionSchema("b"), + new LongDimensionSchema("time"), + new FloatDimensionSchema("c"), + new DoubleDimensionSchema("d") + ), + new AggregatorFactory[] { + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("e", "e") + } + ); + Assert.assertEquals(spec, JSON_MAPPER.readValue(JSON_MAPPER.writeValueAsString(spec), AggregateProjectionSpec.class)); + } + + @Test + public void testInvalidGrouping() + { + Throwable t = Assert.assertThrows( + DruidException.class, + () -> new AggregateProjectionSpec( + "other_projection", + null, + null, + null + ) + ); + Assert.assertEquals("groupingColumns must not be null or empty", t.getMessage()); + + t = Assert.assertThrows( + DruidException.class, + () -> new AggregateProjectionSpec( + "other_projection", + null, + Collections.emptyList(), + null + ) + ); + Assert.assertEquals("groupingColumns must not be null or empty", t.getMessage()); + } + + @Test + public void testEqualsAndHashcode() + { + EqualsVerifier.forClass(AggregateProjectionSpec.class) + .usingGetClass() + .withPrefabValues( + DimensionSchema.class, + new StringDimensionSchema("a"), + new DoubleDimensionSchema("d") + ) + .withIgnoredFields("timeColumnName") + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java index c916a458564c..acc2572664b8 100644 --- a/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java +++ b/processing/src/test/java/org/apache/druid/frame/file/FrameFileTest.java @@ -90,7 +90,7 @@ CursorFactory getCursorFactory() @Override int getRowCount() { - return TestIndex.getNoRollupIncrementalTestIndex().size(); + return TestIndex.getNoRollupIncrementalTestIndex().numRows(); } }, MMAP { diff --git a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java index 4ed2c610525e..000de90a83de 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/FrameProcessorExecutorTest.java @@ -145,11 +145,11 @@ public void test_runFully() throws Exception final ListenableFuture blasterFuture = exec.runFully(blaster, null); final ListenableFuture muxerFuture = exec.runFully(muxer, null); - Assert.assertEquals(index.size(), (long) blasterFuture.get()); - Assert.assertEquals(index.size() * 2, (long) muxerFuture.get()); + Assert.assertEquals(index.numRows(), (long) blasterFuture.get()); + Assert.assertEquals(index.numRows() * 2, (long) muxerFuture.get()); Assert.assertEquals( - index.size() * 2, + index.numRows() * 2, FrameTestUtil.readRowsFromFrameChannel( new ReadableFileFrameChannel(FrameFile.open(outFile, null)), FrameReader.create(cursorFactory.getRowSignature()) diff --git a/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java b/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java index ae748198aea3..baac335f0c42 100644 --- a/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java +++ b/processing/src/test/java/org/apache/druid/frame/processor/test/TestFrameProcessorUtils.java @@ -26,10 +26,7 @@ import org.apache.druid.frame.Frame; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.testutil.FrameSequenceBuilder; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.CursorFactory; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -48,15 +45,11 @@ public static CursorFactory toCursorFactory(List inputRows) { final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - 0, - new TimestampSpec("__time", "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withTimestampSpec(new TimestampSpec("__time", "millis", null)) + .withDimensionsSpec(DimensionsSpec.builder().useSchemaDiscovery(true).build()) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); diff --git a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java index 41834d448fba..a885b2f6df1b 100644 --- a/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java +++ b/processing/src/test/java/org/apache/druid/granularity/QueryGranularityTest.java @@ -33,8 +33,11 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; @@ -55,7 +58,7 @@ /** */ -public class QueryGranularityTest +public class QueryGranularityTest extends InitializedNullHandlingTest { @Test public void testIterableNone() @@ -1056,6 +1059,54 @@ public void testToVirtualColumn() Assert.assertEquals("timestamp_floor(__time,'PT15M')", column.getExpression()); } + @Test + public void testFromVirtualColumn() + { + ExpressionVirtualColumn hourly = Granularities.toVirtualColumn(Granularities.HOUR, "v0"); + ExpressionVirtualColumn day = Granularities.toVirtualColumn(Granularities.DAY, "v0"); + ExpressionVirtualColumn hourlyNonstandardTime = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(__gran, 'PT1H')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn ceilHour = new ExpressionVirtualColumn( + "v0", + "timestamp_ceil(__time, 'PT1M')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn floorWithExpression = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(timestamp_parse(timestamp,null,'UTC'), 'PT1M')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn floorWithTimezone = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(__time, 'PT1M', null,'America/Los_Angeles')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + ExpressionVirtualColumn floorWithOriginTimezone = new ExpressionVirtualColumn( + "v0", + "timestamp_floor(__time, 'PT1M', '2012-01-02T05:00:00.000-08:00','America/Los_Angeles')", + ColumnType.LONG, + TestExprMacroTable.INSTANCE + ); + Assert.assertEquals(Granularities.HOUR, Granularities.fromVirtualColumn(hourly)); + Assert.assertEquals(Granularities.DAY, Granularities.fromVirtualColumn(day)); + Assert.assertEquals(Granularities.HOUR, Granularities.fromVirtualColumn(hourlyNonstandardTime)); + Assert.assertNull(Granularities.fromVirtualColumn(ceilHour)); + Assert.assertNull(Granularities.fromVirtualColumn(floorWithExpression)); + final DateTime origin = DateTimes.of("2012-01-02T05:00:00.000-08:00"); + final DateTimeZone tz = DateTimes.inferTzFromString("America/Los_Angeles"); + final Granularity minuteWithTz = new PeriodGranularity(new Period("PT1M"), null, tz); + final Granularity minuteWithOrigin = new PeriodGranularity(new Period("PT1M"), origin, tz); + Assert.assertEquals(minuteWithTz, Granularities.fromVirtualColumn(floorWithTimezone)); + Assert.assertEquals(minuteWithOrigin, Granularities.fromVirtualColumn(floorWithOriginTimezone)); + } + private void assertBucketStart(final Granularity granularity, final DateTime in, final DateTime expectedInProperTz) { Assert.assertEquals( diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java index 8413c7ea0ed4..4113191a50d1 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerBonusTest.java @@ -88,7 +88,7 @@ public void testOneRowAtATime() throws Exception results = runTimeseriesCount(oneRowIndex); - Assert.assertEquals("index size", 1, oneRowIndex.size()); + Assert.assertEquals("index size", 1, oneRowIndex.numRows()); Assert.assertEquals("result size", 1, results.size()); Assert.assertEquals("result timestamp", DateTimes.of("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); Assert.assertEquals("result count metric", 1, (long) results.get(0).getValue().getLongMetric("rows")); @@ -103,7 +103,7 @@ public void testOneRowAtATime() throws Exception results = runTimeseriesCount(oneRowIndex); - Assert.assertEquals("index size", 2, oneRowIndex.size()); + Assert.assertEquals("index size", 2, oneRowIndex.numRows()); Assert.assertEquals("result size", 1, results.size()); Assert.assertEquals("result timestamp", DateTimes.of("2012-01-01T00:00:00Z"), results.get(0).getTimestamp()); Assert.assertEquals("result count metric", 2, (long) results.get(0).getValue().getLongMetric("rows")); diff --git a/processing/src/test/java/org/apache/druid/segment/AggregateProjectionMetadataTest.java b/processing/src/test/java/org/apache/druid/segment/AggregateProjectionMetadataTest.java new file mode 100644 index 000000000000..49b38da340e0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/AggregateProjectionMetadataTest.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import it.unimi.dsi.fastutil.objects.ObjectAVLTreeSet; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.OrderBy; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.SortedSet; + +public class AggregateProjectionMetadataTest extends InitializedNullHandlingTest +{ + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + AggregateProjectionMetadata spec = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "time", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "time") + ), + Arrays.asList("a", "b", "time", "c", "d"), + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("e", "e") + }, + Arrays.asList( + OrderBy.ascending("a"), + OrderBy.ascending("b"), + OrderBy.ascending("time"), + OrderBy.ascending("c"), + OrderBy.ascending("d") + ) + ), + 12345 + ); + Assert.assertEquals( + spec, + JSON_MAPPER.readValue(JSON_MAPPER.writeValueAsString(spec), AggregateProjectionMetadata.class) + ); + } + + + @Test + public void testComparator() + { + SortedSet metadataBest = new ObjectAVLTreeSet<>(AggregateProjectionMetadata.COMPARATOR); + AggregateProjectionMetadata good = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "good", + "theTime", + VirtualColumns.create(Granularities.toVirtualColumn(Granularities.HOUR, "theTime")), + Arrays.asList("theTime", "a", "b", "c"), + new AggregatorFactory[] { + new CountAggregatorFactory("chocula") + }, + Arrays.asList( + OrderBy.ascending("theTime"), + OrderBy.ascending("a"), + OrderBy.ascending("b"), + OrderBy.ascending("c") + ) + ), + 123 + ); + // same row count, but more aggs more better + AggregateProjectionMetadata better = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "better", + "theTime", + VirtualColumns.create(Granularities.toVirtualColumn(Granularities.HOUR, "theTime")), + Arrays.asList("c", "d", "theTime"), + new AggregatorFactory[] { + new CountAggregatorFactory("chocula"), + new LongSumAggregatorFactory("e", "e") + }, + Arrays.asList( + OrderBy.ascending("c"), + OrderBy.ascending("d"), + OrderBy.ascending("theTime") + ) + ), + 123 + ); + + // small rows is best + AggregateProjectionMetadata best = new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "better", + null, + VirtualColumns.EMPTY, + Arrays.asList("f", "g"), + new AggregatorFactory[0], + Arrays.asList(OrderBy.ascending("f"), OrderBy.ascending("g")) + ), + 10 + ); + metadataBest.add(good); + metadataBest.add(better); + metadataBest.add(best); + Assert.assertEquals(best, metadataBest.first()); + Assert.assertEquals(good, metadataBest.last()); + } + + @Test + public void testEqualsAndHashcode() + { + EqualsVerifier.forClass(AggregateProjectionMetadata.class).usingGetClass().verify(); + } + + @Test + public void testEqualsAndHashcodeSchema() + { + EqualsVerifier.forClass(AggregateProjectionMetadata.Schema.class) + .withIgnoredFields("orderingWithTimeSubstitution", "timeColumnPosition", "granularity") + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index 80b6d23d4b85..2c570981f656 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -26,8 +26,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnType; @@ -494,18 +492,17 @@ public void testNestedColumnIndexerSchemaDiscoveryTypeCoercion() throws IndexSiz long minTimestamp = System.currentTimeMillis(); IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - minTimestamp, - new TimestampSpec(TIME_COL, "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder() - .setDimensions(ImmutableList.of(new AutoTypeColumnSchema(NESTED_COL, ColumnType.STRING))) - .useSchemaDiscovery(true) - .build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withMinTimestamp(minTimestamp) + .withTimestampSpec(new TimestampSpec(TIME_COL, "millis", null)) + .withDimensionsSpec( + DimensionsSpec.builder() + .setDimensions(ImmutableList.of(new AutoTypeColumnSchema(NESTED_COL, ColumnType.STRING))) + .useSchemaDiscovery(true) + .build() + ) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); @@ -699,15 +696,16 @@ private static IncrementalIndex makeIncrementalIndex(long minTimestamp) { IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - minTimestamp, - new TimestampSpec(TIME_COL, "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withMinTimestamp(minTimestamp) + .withTimestampSpec(new TimestampSpec(TIME_COL, "millis", null)) + .withDimensionsSpec( + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .build() + ) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); diff --git a/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java new file mode 100644 index 000000000000..446e714f997a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/CursorFactoryProjectionTest.java @@ -0,0 +1,1238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.Hash; +import it.unimi.dsi.fastutil.objects.ObjectOpenCustomHashSet; +import org.apache.druid.collections.CloseableDefaultBlockingPool; +import org.apache.druid.collections.CloseableStupidPool; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.ListBasedInputRow; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.DoubleDimensionSchema; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.FileUtils; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.firstlast.last.LongLastAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; +import org.apache.druid.query.groupby.GroupingEngine; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.joda.time.DateTime; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +@RunWith(Parameterized.class) +public class CursorFactoryProjectionTest extends InitializedNullHandlingTest +{ + private static final Closer CLOSER = Closer.create(); + private static final DateTime TIMESTAMP = Granularities.DAY.bucket(DateTimes.nowUtc()).getStart(); + + private static final RowSignature ROW_SIGNATURE = RowSignature.builder() + .add("a", ColumnType.STRING) + .add("b", ColumnType.STRING) + .add("c", ColumnType.LONG) + .add("d", ColumnType.DOUBLE) + .build(); + private static final List ROWS = Arrays.asList( + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP, + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "aa", 1L, 1.0) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(2), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "bb", 1L, 1.1, 1.1f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(4), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "cc", 2L, 2.2, 2.2f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(6), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("b", "aa", 3L, 3.3, 3.3f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(8), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("b", "aa", 4L, 4.4, 4.4f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusMinutes(10), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("b", "bb", 5L, 5.5, 5.5f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusHours(1), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "aa", 1L, 1.1, 1.1f) + ), + new ListBasedInputRow( + ROW_SIGNATURE, + TIMESTAMP.plusHours(1).plusMinutes(1), + ROW_SIGNATURE.getColumnNames(), + Arrays.asList("a", "dd", 2L, 2.2, 2.2f) + ) + ); + + private static final List PROJECTIONS = Arrays.asList( + new AggregateProjectionSpec( + "ab_hourly_cd_sum", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("b"), + new LongDimensionSchema("__gran") + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("_c_sum", "c"), + new DoubleSumAggregatorFactory("d", "d") + } + ), + new AggregateProjectionSpec( + "a_hourly_c_sum_with_count_latest", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList( + new LongDimensionSchema("__gran"), + new StringDimensionSchema("a") + ), + new AggregatorFactory[]{ + new CountAggregatorFactory("chocula"), + new LongSumAggregatorFactory("_c_sum", "c"), + new LongLastAggregatorFactory("_c_last", "c", null) + } + ), + new AggregateProjectionSpec( + "bf_daily_c_sum", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.DAY, "__gran") + ), + Arrays.asList( + new LongDimensionSchema("__gran"), + new StringDimensionSchema("b"), + new FloatDimensionSchema("e") + ), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("_c_sum", "c") + } + ), + new AggregateProjectionSpec( + "ab_daily", + null, + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("b") + ), + null + ), + new AggregateProjectionSpec( + "abfoo_daily", + VirtualColumns.create( + new ExpressionVirtualColumn( + "bfoo", + "concat(b, 'foo')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ), + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("bfoo") + ), + null + ), + // cannot really make an 'all' granularity projection, but can do something like floor time to the segment + // granularity interval resulting in a single row + new AggregateProjectionSpec( + "c_sum", + VirtualColumns.create(Granularities.toVirtualColumn(Granularities.DAY, "__gran")), + Collections.singletonList(new LongDimensionSchema("__gran")), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("_c_sum", "c") + } + ) + ); + + private static final List AUTO_PROJECTIONS = PROJECTIONS.stream().map(projection -> { + return new AggregateProjectionSpec( + projection.getName(), + projection.getVirtualColumns(), + projection.getGroupingColumns() + .stream() + .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .collect(Collectors.toList()), + projection.getAggregators() + ); + }).collect(Collectors.toList()); + + @Parameterized.Parameters(name = "name: {0}, sortByDim: {3}, autoSchema: {4}") + public static Collection constructorFeeder() + { + final List constructors = new ArrayList<>(); + final DimensionsSpec.Builder dimensionsBuilder = + DimensionsSpec.builder() + .setDimensions( + Arrays.asList( + new StringDimensionSchema("a"), + new StringDimensionSchema("b"), + new LongDimensionSchema("c"), + new DoubleDimensionSchema("d"), + new FloatDimensionSchema("e") + ) + ); + DimensionsSpec dimsTimeOrdered = dimensionsBuilder.build(); + DimensionsSpec dimsOrdered = dimensionsBuilder.setForceSegmentSortByTime(false).build(); + + + List autoDims = dimsOrdered.getDimensions() + .stream() + .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .collect(Collectors.toList()); + for (boolean incremental : new boolean[]{true, false}) { + for (boolean sortByDim : new boolean[]{true, false}) { + for (boolean autoSchema : new boolean[]{true, false}) { + final DimensionsSpec dims; + if (sortByDim) { + if (autoSchema) { + dims = dimsOrdered.withDimensions(autoDims); + } else { + dims = dimsOrdered; + } + } else { + if (autoSchema) { + dims = dimsTimeOrdered.withDimensions(autoDims); + } else { + dims = dimsTimeOrdered; + } + } + if (incremental) { + IncrementalIndex index = CLOSER.register(makeBuilder(dims, autoSchema).buildIncrementalIndex()); + constructors.add(new Object[]{ + "incrementalIndex", + new IncrementalIndexCursorFactory(index), + new IncrementalIndexTimeBoundaryInspector(index), + sortByDim, + autoSchema + }); + } else { + QueryableIndex index = CLOSER.register(makeBuilder(dims, autoSchema).buildMMappedIndex()); + constructors.add(new Object[]{ + "queryableIndex", + new QueryableIndexCursorFactory(index), + QueryableIndexTimeBoundaryInspector.create(index), + sortByDim, + autoSchema + }); + } + } + } + } + return constructors; + } + + @AfterClass + public static void cleanup() throws IOException + { + CLOSER.close(); + } + + + public final CursorFactory projectionsCursorFactory; + public final TimeBoundaryInspector projectionsTimeBoundaryInspector; + + private final GroupingEngine groupingEngine; + private final TimeseriesQueryEngine timeseriesEngine; + + private final NonBlockingPool nonBlockingPool; + public final boolean sortByDim; + public final boolean autoSchema; + + @Rule + public final CloserRule closer = new CloserRule(false); + + public CursorFactoryProjectionTest( + String name, + CursorFactory projectionsCursorFactory, + TimeBoundaryInspector projectionsTimeBoundaryInspector, + boolean sortByDim, + boolean autoSchema + ) + { + this.projectionsCursorFactory = projectionsCursorFactory; + this.projectionsTimeBoundaryInspector = projectionsTimeBoundaryInspector; + this.sortByDim = sortByDim; + this.autoSchema = autoSchema; + this.nonBlockingPool = closer.closeLater( + new CloseableStupidPool<>( + "GroupByQueryEngine-bufferPool", + () -> ByteBuffer.allocate(50000) + ) + ); + this.groupingEngine = new GroupingEngine( + new DruidProcessingConfig(), + GroupByQueryConfig::new, + new GroupByResourcesReservationPool( + closer.closeLater( + new CloseableDefaultBlockingPool<>( + () -> ByteBuffer.allocate(50000), + 5 + ) + ), + new GroupByQueryConfig() + ), + TestHelper.makeJsonMapper(), + TestHelper.makeSmileMapper(), + (query, future) -> { + } + ); + this.timeseriesEngine = new TimeseriesQueryEngine(nonBlockingPool); + } + + @Test + public void testProjectionSelectionTwoDims() + { + // this query can use the projection with 2 dims, which has 7 rows instead of the total of 8 + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("b") + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(6, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(6, results.size()); + if (projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + if (autoSchema) { + Assert.assertArrayEquals(new Object[]{"b", "bb"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa"}, results.get(5).getArray()); + } else { + Assert.assertArrayEquals(new Object[]{"a", "dd"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb"}, results.get(5).getArray()); + } + } else { + Assert.assertArrayEquals(new Object[]{"a", "aa"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb"}, results.get(5).getArray()); + } + } + + @Test + public void testProjectionSelectionTwoDimsVirtual() + { + // this query can use the projection with 2 dims, which has 7 rows instead of the total of 8 + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("v0") + .setVirtualColumns( + new ExpressionVirtualColumn( + "v0", + "concat(b, 'foo')", + ColumnType.STRING, + TestExprMacroTable.INSTANCE + ) + ) + .setLimitSpec( + new DefaultLimitSpec( + Arrays.asList( + new OrderByColumnSpec("a", OrderByColumnSpec.Direction.ASCENDING, StringComparators.LEXICOGRAPHIC), + new OrderByColumnSpec("v0", OrderByColumnSpec.Direction.ASCENDING, StringComparators.LEXICOGRAPHIC) + ), + 10 + ) + ) + .setContext(ImmutableMap.of(QueryContexts.USE_PROJECTION, "abfoo_daily")) + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(6, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(6, results.size()); + if (projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + // testing ordering of stuff is kind of tricky at this level... + if (autoSchema) { + Assert.assertArrayEquals(new Object[]{"b", "bbfoo"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ddfoo"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aafoo"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ccfoo"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bbfoo"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aafoo"}, results.get(5).getArray()); + } else { + Assert.assertArrayEquals(new Object[]{"a", "ddfoo"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aafoo"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aafoo"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ccfoo"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bbfoo"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bbfoo"}, results.get(5).getArray()); + } + } else { + Assert.assertArrayEquals(new Object[]{"a", "aafoo"}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bbfoo"}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ccfoo"}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "ddfoo"}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aafoo"}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bbfoo"}, results.get(5).getArray()); + } + } + + @Test + public void testProjectionSelectionTwoDimsCount() + { + // cannot use a projection since count is not defined + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("b") + .addAggregator(new CountAggregatorFactory("count")) + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(6, results.size()); + if (projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + if (autoSchema) { + Assert.assertArrayEquals(new Object[]{"b", "aa", 2L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc", 1L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb", 1L}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb", 1L}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd", 1L}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa", 2L}, results.get(5).getArray()); + } else { + Assert.assertArrayEquals(new Object[]{"a", "dd", 1L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "aa", 2L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb", 1L}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa", 2L}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc", 1L}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb", 1L}, results.get(5).getArray()); + } + } else { + Assert.assertArrayEquals(new Object[]{"a", "aa", 2L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "bb", 1L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "cc", 1L}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "aa", 2L}, results.get(3).getArray()); + Assert.assertArrayEquals(new Object[]{"b", "bb", 1L}, results.get(4).getArray()); + Assert.assertArrayEquals(new Object[]{"a", "dd", 1L}, results.get(5).getArray()); + } + } + + @Test + public void testProjectionSelectionTwoDimsCountForce() + { + // cannot use a projection since count is not defined + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("b") + .addAggregator(new CountAggregatorFactory("count")) + .setContext(ImmutableMap.of(QueryContexts.FORCE_PROJECTION, true)) + .build(); + + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + + Throwable t = Assert.assertThrows( + DruidException.class, + () -> projectionsCursorFactory.makeCursorHolder(buildSpec) + ); + Assert.assertEquals("Force projections specified, but none satisfy query", t.getMessage()); + } + + @Test + public void testProjectionSkipContext() + { + // setting context flag to skip projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongLastAggregatorFactory("c_last", "c", null)) + .setContext(ImmutableMap.of(QueryContexts.NO_PROJECTIONS, true)) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + // has to scan full 8 rows because context ensures projections not used + Assert.assertEquals(8, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals( + new Object[]{"a", 7L, Pair.of(TIMESTAMP.plusHours(1).plusMinutes(1).getMillis(), 2L)}, + results.get(0).getArray() + ); + Assert.assertArrayEquals( + new Object[]{"b", 12L, Pair.of(TIMESTAMP.plusMinutes(10).getMillis(), 5L)}, + results.get(1).getArray() + ); + } + + @Test + public void testProjectionSingleDim() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongLastAggregatorFactory("c_last", "c", null)) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals( + new Object[]{"a", 7L, Pair.of(TIMESTAMP.plusHours(1).plusMinutes(1).getMillis(), 2L)}, + results.get(0).getArray() + ); + Assert.assertArrayEquals( + new Object[]{"b", 12L, Pair.of(TIMESTAMP.plusMinutes(10).getMillis(), 5L)}, + results.get(1).getArray() + ); + } + + @Test + public void testProjectionSingleDimFilter() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .setDimFilter(new EqualityFilter("a", ColumnType.STRING, "a", null)) + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongLastAggregatorFactory("c_last", "c", null)) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(2, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(1, results.size()); + Assert.assertArrayEquals( + new Object[]{"a", 7L, Pair.of(TIMESTAMP.plusHours(1).plusMinutes(1).getMillis(), 2L)}, + results.get(0).getArray() + ); + } + + @Test + public void testProjectionSingleDimCount() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new CountAggregatorFactory("cnt")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", 7L, 5L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", 12L, 3L}, results.get(1).getArray()); + } + + @Test + public void testProjectionSingleDimMultipleSameAggs() + { + // test can use the single dimension projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new LongSumAggregatorFactory("c_sum_2", "c")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", 7L, 7L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", 12L, 12L}, results.get(1).getArray()); + } + + @Test + public void testQueryGranularityFinerThanProjectionGranularity() + { + final GroupByQuery.Builder queryBuilder = + GroupByQuery.builder() + .setDataSource("test") + .setInterval(Intervals.ETERNITY) + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")); + if (sortByDim) { + queryBuilder.setVirtualColumns(Granularities.toVirtualColumn(Granularities.MINUTE, "__gran")) + .setDimensions( + DefaultDimensionSpec.of("__gran", ColumnType.LONG), + DefaultDimensionSpec.of("a") + ) + .setGranularity(Granularities.ALL); + } else { + queryBuilder.addDimension("a") + .setGranularity(Granularities.MINUTE); + } + final GroupByQuery query = queryBuilder.build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(8, results.size()); + + if (sortByDim && projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + // this sorts funny when not time ordered + Set resultsInNoParticularOrder = makeArrayResultSet(); + resultsInNoParticularOrder.addAll( + ROWS.stream() + .map(x -> new Object[]{x.getTimestamp().getMillis(), x.getRaw("a"), x.getRaw("c")}) + .collect(Collectors.toList()) + ); + for (ResultRow row : results) { + Assert.assertTrue(resultsInNoParticularOrder.contains(row.getArray())); + } + } else { + Assert.assertArrayEquals( + new Object[]{ROWS.get(0).getTimestamp().getMillis(), "a", 1L}, + results.get(0).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(1).getTimestamp().getMillis(), "a", 1L}, + results.get(1).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(2).getTimestamp().getMillis(), "a", 2L}, + results.get(2).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(3).getTimestamp().getMillis(), "b", 3L}, + results.get(3).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(4).getTimestamp().getMillis(), "b", 4L}, + results.get(4).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(5).getTimestamp().getMillis(), "b", 5L}, + results.get(5).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(6).getTimestamp().getMillis(), "a", 1L}, + results.get(6).getArray() + ); + Assert.assertArrayEquals( + new Object[]{ROWS.get(7).getTimestamp().getMillis(), "a", 2L}, + results.get(7).getArray() + ); + } + } + + @Test + public void testQueryGranularityFitsProjectionGranularity() + { + final GroupByQuery.Builder queryBuilder = + GroupByQuery.builder() + .setDataSource("test") + .setInterval(Intervals.ETERNITY) + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")); + if (sortByDim) { + queryBuilder.setGranularity(Granularities.ALL) + .setDimensions( + DefaultDimensionSpec.of("__gran", ColumnType.LONG), + DefaultDimensionSpec.of("a") + ) + .setVirtualColumns(Granularities.toVirtualColumn(Granularities.HOUR, "__gran")); + } else { + queryBuilder.addDimension("a") + .setGranularity(Granularities.HOUR); + } + final GroupByQuery query = queryBuilder.build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(3, results.size()); + if (sortByDim && projectionsCursorFactory instanceof QueryableIndexCursorFactory) { + Set resultsInNoParticularOrder = makeArrayResultSet( + new Object[]{TIMESTAMP.getMillis(), "a", 4L}, + new Object[]{TIMESTAMP.getMillis(), "b", 12L}, + new Object[]{TIMESTAMP.plusHours(1).getMillis(), "a", 3L} + ); + for (ResultRow row : results) { + Assert.assertTrue(resultsInNoParticularOrder.contains(row.getArray())); + } + } else { + Assert.assertArrayEquals(new Object[]{TIMESTAMP.getMillis(), "a", 4L}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.getMillis(), "b", 12L}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1).getMillis(), "a", 3L}, results.get(2).getArray()); + } + } + + @Test + public void testProjectionSelectionMissingAggregatorButHasAggregatorInput() + { + // d is present as a column on the projection, but since its an aggregate projection we cannot use it + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("b") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new FloatSumAggregatorFactory("e_sum", "e")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(4, results.size()); + Assert.assertArrayEquals(new Object[]{"aa", 9L, NullHandling.defaultFloatValue()}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"bb", 6L, NullHandling.defaultFloatValue()}, results.get(1).getArray()); + Assert.assertArrayEquals(new Object[]{"cc", 2L, NullHandling.defaultFloatValue()}, results.get(2).getArray()); + Assert.assertArrayEquals(new Object[]{"dd", 2L, NullHandling.defaultFloatValue()}, results.get(3).getArray()); + } + + @Test + public void testProjectionSelectionMissingAggregatorAndAggregatorInput() + { + // since d isn't present on the smaller projection, cant use it, but can still use the larger projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new DoubleSumAggregatorFactory("d_sum", "d")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(7, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", 7L, 7.6000000000000005}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", 12L, 13.2}, results.get(1).getArray()); + } + + @Test + public void testProjectionSelectionMissingColumnOnBaseTableToo() + { + // since d isn't present on the smaller projection, cant use it, but can still use the larger projection + final GroupByQuery query = + GroupByQuery.builder() + .setDataSource("test") + .setGranularity(Granularities.ALL) + .setInterval(Intervals.ETERNITY) + .addDimension("a") + .addDimension("z") + .addAggregator(new LongSumAggregatorFactory("c_sum", "c")) + .addAggregator(new DoubleSumAggregatorFactory("d_sum", "d")) + .build(); + final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(7, rowCount); + } + + final Sequence resultRows = groupingEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + nonBlockingPool, + null + ); + + final List results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + Assert.assertArrayEquals(new Object[]{"a", null, 7L, 7.6000000000000005}, results.get(0).getArray()); + Assert.assertArrayEquals(new Object[]{"b", null, 12L, 13.2}, results.get(1).getArray()); + } + + private static IndexBuilder makeBuilder(DimensionsSpec dimensionsSpec, boolean autoSchema) + { + File tmp = FileUtils.createTempDir(); + CLOSER.register(tmp::delete); + return IndexBuilder.create() + .tmpDir(tmp) + .schema( + IncrementalIndexSchema.builder() + .withDimensionsSpec(dimensionsSpec) + .withRollup(false) + .withMinTimestamp(TIMESTAMP.getMillis()) + .withProjections(autoSchema ? AUTO_PROJECTIONS : PROJECTIONS) + .build() + ) + .rows(ROWS); + } + + @Test + public void testTimeseriesQueryGranularityFitsProjectionGranularity() + { + Assume.assumeFalse(sortByDim); + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.HOUR) + .aggregators(new LongSumAggregatorFactory("c_sum", "c")) + .build(); + + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(3, rowCount); + } + + final Sequence> resultRows = timeseriesEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + null + ); + + final List> results = resultRows.toList(); + Assert.assertEquals(2, results.size()); + final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES); + Assert.assertArrayEquals(new Object[]{TIMESTAMP, 16L}, getResultArray(results.get(0), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1), 3L}, getResultArray(results.get(1), querySignature)); + } + + @Test + public void testTimeseriesQueryGranularityAllFitsProjectionGranularity() + { + Assume.assumeFalse(sortByDim); + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.ALL) + .aggregators(new LongSumAggregatorFactory("c_sum", "c")) + .context(ImmutableMap.of(QueryContexts.USE_PROJECTION, "c_sum")) + .build(); + + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(1, rowCount); + } + + final Sequence> resultRows = timeseriesEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + null + ); + + final List> results = resultRows.toList(); + Assert.assertEquals(1, results.size()); + final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES); + Assert.assertArrayEquals(new Object[]{TIMESTAMP, 19L}, getResultArray(results.get(0), querySignature)); + } + + @Test + public void testTimeseriesQueryGranularityFinerThanProjectionGranularity() + { + Assume.assumeFalse(sortByDim); + final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.MINUTE) + .aggregators(new LongSumAggregatorFactory("c_sum", "c")) + .context(ImmutableMap.of(TimeseriesQuery.SKIP_EMPTY_BUCKETS, true)) + .build(); + + final CursorBuildSpec buildSpec = TimeseriesQueryEngine.makeCursorBuildSpec(query, null); + try (final CursorHolder cursorHolder = projectionsCursorFactory.makeCursorHolder(buildSpec)) { + final Cursor cursor = cursorHolder.asCursor(); + int rowCount = 0; + while (!cursor.isDone()) { + rowCount++; + cursor.advance(); + } + Assert.assertEquals(8, rowCount); + } + + final Sequence> resultRows = timeseriesEngine.process( + query, + projectionsCursorFactory, + projectionsTimeBoundaryInspector, + null + ); + + final List> results = resultRows.toList(); + Assert.assertEquals(8, results.size()); + final RowSignature querySignature = query.getResultSignature(RowSignature.Finalization.YES); + Assert.assertArrayEquals(new Object[]{TIMESTAMP, 1L}, getResultArray(results.get(0), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(2), 1L}, getResultArray(results.get(1), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(4), 2L}, getResultArray(results.get(2), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(6), 3L}, getResultArray(results.get(3), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(8), 4L}, getResultArray(results.get(4), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusMinutes(10), 5L}, getResultArray(results.get(5), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1), 1L}, getResultArray(results.get(6), querySignature)); + Assert.assertArrayEquals(new Object[]{TIMESTAMP.plusHours(1).plusMinutes(1), 2L}, getResultArray(results.get(7), querySignature)); + } + + + private static Set makeArrayResultSet() + { + Set resultsInNoParticularOrder = new ObjectOpenCustomHashSet<>( + new Hash.Strategy() + { + @Override + public int hashCode(Object[] o) + { + return Arrays.hashCode(o); + } + + @Override + public boolean equals(Object[] a, Object[] b) + { + return Arrays.deepEquals(a, b); + } + } + ); + return resultsInNoParticularOrder; + } + + private static Set makeArrayResultSet(Object[]... values) + { + Set resultsInNoParticularOrder = makeArrayResultSet(); + resultsInNoParticularOrder.addAll(Arrays.asList(values)); + return resultsInNoParticularOrder; + } + + private static Object[] getResultArray(Result result, RowSignature rowSignature) + { + final Object[] rowArray = new Object[rowSignature.size()]; + for (int i = 0; i < rowSignature.size(); i++) { + if (i == 0) { + rowArray[i] = result.getTimestamp(); + } else { + rowArray[i] = result.getValue().getMetric(rowSignature.getColumnName(i)); + } + } + return rowArray; + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java index 52c129379fc5..262ecd500155 100644 --- a/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CursorHolderPreaggTest.java @@ -30,6 +30,7 @@ import org.apache.druid.query.Druids; import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; @@ -59,6 +60,7 @@ import java.io.Closeable; import java.nio.ByteBuffer; import java.util.List; +import java.util.stream.Collectors; public class CursorHolderPreaggTest extends InitializedNullHandlingTest { @@ -137,6 +139,15 @@ public boolean isPreAggregated() return true; } + @Nullable + @Override + public List getAggregatorsForPreAggregated() + { + return spec.getAggregators() + .stream().map(AggregatorFactory::getCombiningFactory) + .collect(Collectors.toList()); + } + @Override public void close() { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java index 703de4439e40..92bc826bb575 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java @@ -273,7 +273,7 @@ private ColumnHolder deserializeColumn( ColumnDescriptor serde = mapper.readValue( IndexIO.SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class ); - return serde.read(byteBuffer, columnConfig, smooshedFiles); + return serde.read(byteBuffer, columnConfig, smooshedFiles, null); } } } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 5c519f884586..7faa31d8e53f 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -410,7 +410,8 @@ public void testPersistWithSegmentMetadata() throws Exception null, Granularities.NONE, Boolean.TRUE, - Cursors.ascendingTimeOrder() + Cursors.ascendingTimeOrder(), + null ), index.getMetadata() ); diff --git a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java index b158480b22d6..5d06d2ab8c25 100644 --- a/processing/src/test/java/org/apache/druid/segment/MetadataTest.java +++ b/processing/src/test/java/org/apache/druid/segment/MetadataTest.java @@ -22,12 +22,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.OrderBy; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.firstlast.last.LongLastAggregatorFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; @@ -40,7 +45,7 @@ /** * */ -public class MetadataTest +public class MetadataTest extends InitializedNullHandlingTest { @Test public void testSerde() throws Exception @@ -57,6 +62,7 @@ public void testSerde() throws Exception null, Granularities.ALL, Boolean.FALSE, + null, null ); @@ -83,13 +89,32 @@ public void testMerge() AggregatorFactory[] aggs = new AggregatorFactory[]{ new LongMaxAggregatorFactory("n", "f") }; + List projectionSpecs = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "__gran"), + new AggregatorFactory[]{ + new LongLastAggregatorFactory("atLongLast", "d", null) + }, + makeOrderBy("a", "b", "__gran") + ), + 1234 + ) + ); + final Metadata m1 = new Metadata( Collections.singletonMap("k", "v"), aggs, new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE, - null + null, + projectionSpecs ); final Metadata m2 = new Metadata( @@ -98,7 +123,8 @@ public void testMerge() new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE, - null + null, + projectionSpecs ); final Metadata m3 = new Metadata( @@ -107,7 +133,8 @@ public void testMerge() new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.TRUE, - null + null, + projectionSpecs ); final Metadata merged = new Metadata( @@ -118,7 +145,8 @@ public void testMerge() new TimestampSpec("ds", "auto", null), Granularities.ALL, Boolean.FALSE, - Cursors.ascendingTimeOrder() + Cursors.ascendingTimeOrder(), + projectionSpecs ); Assert.assertEquals(merged, Metadata.merge(ImmutableList.of(m1, m2), null)); @@ -129,7 +157,15 @@ public void testMerge() metadataToBeMerged.add(null); final Metadata merged2 = - new Metadata(Collections.singletonMap("k", "v"), null, null, null, null, Cursors.ascendingTimeOrder()); + new Metadata( + Collections.singletonMap("k", "v"), + null, + null, + null, + null, + Cursors.ascendingTimeOrder(), + projectionSpecs + ); Assert.assertEquals(merged2, Metadata.merge(metadataToBeMerged, null)); @@ -139,7 +175,15 @@ public void testMerge() }; final Metadata merged3 = - new Metadata(Collections.singletonMap("k", "v"), explicitAggs, null, null, null, Cursors.ascendingTimeOrder()); + new Metadata( + Collections.singletonMap("k", "v"), + explicitAggs, + null, + null, + null, + Cursors.ascendingTimeOrder(), + projectionSpecs + ); Assert.assertEquals( merged3, @@ -152,7 +196,8 @@ public void testMerge() new TimestampSpec("ds", "auto", null), Granularities.ALL, null, - Cursors.ascendingTimeOrder() + Cursors.ascendingTimeOrder(), + projectionSpecs ); Assert.assertEquals( merged4, @@ -219,6 +264,107 @@ public void testMergeOrderings() ); } + @Test + public void testMergeProjectionsUnexpectedMismatch() + { + List p1 = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "__gran"), + new AggregatorFactory[]{ + new LongLastAggregatorFactory("atLongLast", "d", null) + }, + makeOrderBy("a", "b", "__gran") + ), + 654321 + ) + ); + + List p2 = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "_gran"), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("longSum", "d") + }, + makeOrderBy("a", "b", "__gran") + ), + 1234 + ) + ); + + List p3 = ImmutableList.of( + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.HOUR, "__gran") + ), + Arrays.asList("a", "b", "__gran"), + new AggregatorFactory[]{ + new LongLastAggregatorFactory("atLongLast", "d", null) + }, + makeOrderBy("a", "b", "__gran") + ), + 12121 + ), + new AggregateProjectionMetadata( + new AggregateProjectionMetadata.Schema( + "some_projection2", + "__gran", + VirtualColumns.create( + Granularities.toVirtualColumn(Granularities.DAY, "__gran") + ), + Arrays.asList("__gran", "a"), + new AggregatorFactory[]{ + new LongSumAggregatorFactory("longSum", "d") + }, + makeOrderBy("__gran", "a") + ), + 555 + ) + ); + + Throwable t = Assert.assertThrows( + DruidException.class, + () -> Metadata.validateProjections(Arrays.asList(p1, p2)) + ); + MatcherAssert.assertThat( + t.getMessage(), + CoreMatchers.startsWith("Unable to merge projections: mismatched projections") + ); + + t = Assert.assertThrows( + DruidException.class, + () -> Metadata.validateProjections(Arrays.asList(p1, p3)) + ); + + MatcherAssert.assertThat( + t.getMessage(), + CoreMatchers.startsWith("Unable to merge projections: mismatched projections count") + ); + + t = Assert.assertThrows( + DruidException.class, + () -> Metadata.validateProjections(Arrays.asList(p1, null)) + ); + MatcherAssert.assertThat( + t.getMessage(), + CoreMatchers.startsWith("Unable to merge projections: some projections were null") + ); + } + private static List makeOrderBy(final String... columnNames) { return Arrays.stream(columnNames).map(OrderBy::ascending).collect(Collectors.toList()); diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java index 2e9deab42b49..9fc9fc0f578d 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerV4Test.java @@ -26,8 +26,6 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.column.ColumnType; @@ -478,15 +476,16 @@ private static IncrementalIndex makeIncrementalIndex(long minTimestamp) { IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( - new IncrementalIndexSchema( - minTimestamp, - new TimestampSpec(TIME_COL, "millis", null), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.builder().useSchemaDiscovery(true).build(), - new AggregatorFactory[0], - false - ) + IncrementalIndexSchema.builder() + .withMinTimestamp(minTimestamp) + .withTimestampSpec(new TimestampSpec(TIME_COL, "millis", null)) + .withDimensionsSpec( + DimensionsSpec.builder() + .useSchemaDiscovery(true) + .build() + ) + .withRollup(false) + .build() ) .setMaxRowCount(1000) .build(); diff --git a/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java b/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java index 980b23e140c3..69dbea9797b2 100644 --- a/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/V9IndexLoaderTest.java @@ -31,6 +31,7 @@ import org.junit.Assert; import org.junit.Test; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -128,7 +129,8 @@ ColumnHolder deserializeColumn( String columnName, ObjectMapper mapper, ByteBuffer byteBuffer, - SmooshedFileMapper smooshedFiles + SmooshedFileMapper smooshedFiles, + @Nullable ColumnHolder parentColumn ) throws IOException { if (COUNT_COLUMN.equals(columnName)) { @@ -138,7 +140,8 @@ ColumnHolder deserializeColumn( columnName, mapper, byteBuffer, - smooshedFiles + smooshedFiles, + parentColumn ); } } diff --git a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java index 9c0e3d8f158c..b29a571389ce 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/IncrementalIndexTest.java @@ -233,7 +233,7 @@ public void testCaseSensitivity() throws Exception populateIndex(timestamp, index); Assert.assertEquals(Arrays.asList("__time", "dim1", "dim2"), index.getDimensionNames(true)); Assert.assertEquals(Arrays.asList("dim1", "dim2"), index.getDimensionNames(false)); - Assert.assertEquals(2, index.size()); + Assert.assertEquals(2, index.numRows()); final Iterator rows = index.iterator(); Row row = rows.next(); @@ -299,7 +299,7 @@ public void testFilteredAggregators() throws Exception ), index.getMetricNames() ); - Assert.assertEquals(2, index.size()); + Assert.assertEquals(2, index.numRows()); final Iterator rows = index.iterator(); Row row = rows.next(); @@ -687,7 +687,7 @@ public void testDynamicSchemaRollup() throws IndexSizeExceededException ) ); - Assert.assertEquals(2, index.size()); + Assert.assertEquals(2, index.numRows()); } @Test @@ -727,7 +727,7 @@ public void testSchemaRollupWithRowWithExistingMetricsAndWithoutMetric() throws ) ); - Assert.assertEquals(index.isRollup() ? 1 : 4, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 4, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -795,7 +795,7 @@ public void testSchemaRollupWithRowWithExistingMetricsAndWithoutMetricUsingAggre ) ); - Assert.assertEquals(index.isRollup() ? 1 : 4, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 4, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -848,7 +848,7 @@ public void testSchemaRollupWithRowWithOnlyExistingMetrics() throws IndexSizeExc ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -901,7 +901,7 @@ public void testSchemaRollupWithRowsWithNoMetrics() throws IndexSizeExceededExce ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -951,7 +951,7 @@ public void testSchemaRollupWithRowWithMixedTypeMetrics() throws IndexSizeExceed ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { @@ -999,7 +999,7 @@ public void testSchemaRollupWithRowsWithNonRolledUpSameColumnName() throws Index ) ); - Assert.assertEquals(index.isRollup() ? 1 : 2, index.size()); + Assert.assertEquals(index.isRollup() ? 1 : 2, index.numRows()); Iterator iterator = index.iterator(); int rowCount = 0; while (iterator.hasNext()) { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index ed4ff921a9c5..aa28106073ea 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -542,7 +542,8 @@ public static Collection makeConstructors() .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final IncrementalIndex index = input.buildIncrementalIndex(); @@ -570,7 +571,8 @@ public static Collection makeConstructors() .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final QueryableIndex index = input.buildMMappedIndex(); @@ -599,7 +601,8 @@ public static Collection makeConstructors() .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ) // if 1 row per segment some of the columns have null values for the row which causes 'auto' @@ -676,7 +679,8 @@ public static Collection makeConstructors() .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED); @@ -699,7 +703,8 @@ public static Collection makeConstructors() .collect(Collectors.toList()) ), schema.getMetrics(), - schema.isRollup() + schema.isRollup(), + schema.getProjections() ) ); final FrameSegment segment = input.buildFrameSegment(FrameType.COLUMNAR); diff --git a/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java index 0cb5f25ef730..dd88367e7832 100644 --- a/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java @@ -744,6 +744,6 @@ public void testToIndex() .build(); dataGenerator.addToIndex(index, 100); - Assert.assertEquals(100, index.size()); + Assert.assertEquals(100, index.numRows()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactoryTest.java similarity index 83% rename from processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java rename to processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactoryTest.java index 87c60cb74207..36abb79be8c9 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexCursorFactoryTest.java @@ -24,8 +24,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.druid.collections.CloseableDefaultBlockingPool; import org.apache.druid.collections.CloseableStupidPool; -import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; @@ -54,9 +55,9 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByResourcesReservationPool; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; -import org.apache.druid.query.groupby.epinephelinae.GroupByQueryEngine; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.query.topn.TopNQueryEngine; import org.apache.druid.query.topn.TopNResultValue; @@ -70,6 +71,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector; +import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.SelectorFilter; @@ -96,12 +98,19 @@ import java.util.Set; /** + * */ @RunWith(Parameterized.class) -public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingTest +public class IncrementalIndexCursorFactoryTest extends InitializedNullHandlingTest { public final IncrementalIndexCreator indexCreator; + private final GroupingEngine groupingEngine; + private final TopNQueryEngine topnQueryEngine; + + private final NonBlockingPool nonBlockingPool; + + /** * If true, sort by [billy, __time]. If false, sort by [__time]. */ @@ -110,7 +119,8 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT @Rule public final CloserRule closer = new CloserRule(false); - public IncrementalIndexStorageAdapterTest(String indexType, boolean sortByDim) throws JsonProcessingException + public IncrementalIndexCursorFactoryTest(String indexType, boolean sortByDim) + throws JsonProcessingException { BuiltInTypesModule.registerHandlersAndSerde(); this.sortByDim = sortByDim; @@ -144,6 +154,31 @@ public IncrementalIndexStorageAdapterTest(String indexType, boolean sortByDim) t } ) ); + + nonBlockingPool = closer.closeLater( + new CloseableStupidPool<>( + "GroupByQueryEngine-bufferPool", + () -> ByteBuffer.allocate(50000) + ) + ); + groupingEngine = new GroupingEngine( + new DruidProcessingConfig(), + GroupByQueryConfig::new, + new GroupByResourcesReservationPool( + closer.closeLater( + new CloseableDefaultBlockingPool<>( + () -> ByteBuffer.allocate(50000), + 5 + ) + ), + new GroupByQueryConfig() + ), + TestHelper.makeJsonMapper(), + TestHelper.makeSmileMapper(), + (query, future) -> { + } + ); + topnQueryEngine = new TopNQueryEngine(nonBlockingPool); } @Parameterized.Parameters(name = "{index}: {0}, sortByDim: {1}") @@ -183,37 +218,24 @@ public void testSanity() throws Exception .addAggregator(new LongSumAggregatorFactory("cnt", "cnt")) .addOrderByColumn("billy") .build(); - final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "GroupByQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) - ); - ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) - ) { - final Sequence rows = GroupByQueryEngine.process( - query, - new IncrementalIndexTimeBoundaryInspector(index), - cursorHolder, - buildSpec, - processingBuffer.get(), - null, - new GroupByQueryConfig(), - new DruidProcessingConfig() - ); + final Sequence rows = groupingEngine.process( + query, + cursorFactory, + new IncrementalIndexTimeBoundaryInspector(index), + nonBlockingPool, + null + ); - final List results = rows.toList(); + final List results = rows.toList(); - Assert.assertEquals(2, results.size()); + Assert.assertEquals(2, results.size()); - ResultRow row = results.get(0); - Assert.assertArrayEquals(new Object[]{NullHandling.defaultStringValue(), "bo", 1L}, row.getArray()); + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{NullHandling.defaultStringValue(), "bo", 1L}, row.getArray()); - row = results.get(1); - Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); - } + row = results.get(1); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } @Test @@ -260,39 +282,26 @@ public void testObjectColumnSelectorOnVaryingColumnSchema() throws Exception .addOrderByColumn("billy") .build(); final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "GroupByQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) - ); - ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) - ) { - final Sequence rows = GroupByQueryEngine.process( - query, - new IncrementalIndexTimeBoundaryInspector(index), - cursorHolder, - buildSpec, - processingBuffer.get(), - null, - new GroupByQueryConfig(), - new DruidProcessingConfig() - ); + final Sequence rows = groupingEngine.process( + query, + cursorFactory, + new IncrementalIndexTimeBoundaryInspector(index), + nonBlockingPool, + null + ); - final List results = rows.toList(); + final List results = rows.toList(); - Assert.assertEquals(2, results.size()); + Assert.assertEquals(2, results.size()); - ResultRow row = results.get(0); - Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L, 2.0}, row.getArray()); + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L, 2.0}, row.getArray()); - row = results.get(1); - Assert.assertArrayEquals( - new Object[]{"hip", "hop", 1L, 6.0}, - row.getArray() - ); - } + row = results.get(1); + Assert.assertArrayEquals( + new Object[]{"hip", "hop", 1L, 6.0}, + row.getArray() + ); } @Test @@ -371,33 +380,24 @@ public void testSingleValueTopN() throws IOException ) ); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "TopNQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) + final Iterable> results = topnQueryEngine + .query( + new TopNQueryBuilder() + .dataSource("test") + .granularity(Granularities.ALL) + .intervals(Collections.singletonList(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))) + .dimension("sally") + .metric("cnt") + .threshold(10) + .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) + .build(), + new IncrementalIndexSegment(index, SegmentId.dummy("test")), + null ) - ) { - TopNQueryEngine engine = new TopNQueryEngine(pool); - - final Iterable> results = engine - .query( - new TopNQueryBuilder() - .dataSource("test") - .granularity(Granularities.ALL) - .intervals(Collections.singletonList(new Interval(DateTimes.EPOCH, DateTimes.nowUtc()))) - .dimension("sally") - .metric("cnt") - .threshold(10) - .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) - .build(), - new IncrementalIndexSegment(index, SegmentId.dummy("test")), - null - ) - .toList(); + .toList(); - Assert.assertEquals(1, Iterables.size(results)); - Assert.assertEquals(1, results.iterator().next().getValue().getValue().size()); - } + Assert.assertEquals(1, Iterables.size(results)); + Assert.assertEquals(1, results.iterator().next().getValue().getValue().size()); } @Test @@ -430,34 +430,21 @@ public void testFilterByNull() throws Exception .setDimFilter(DimFilters.dimEquals("sally", (String) null)) .build(); final IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(index); - final CursorBuildSpec buildSpec = GroupingEngine.makeCursorBuildSpec(query, null); - try ( - CloseableStupidPool pool = new CloseableStupidPool<>( - "GroupByQueryEngine-bufferPool", - () -> ByteBuffer.allocate(50000) - ); - ResourceHolder processingBuffer = pool.take(); - final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec) - ) { - final Sequence rows = GroupByQueryEngine.process( - query, - new IncrementalIndexTimeBoundaryInspector(index), - cursorHolder, - buildSpec, - processingBuffer.get(), - null, - new GroupByQueryConfig(), - new DruidProcessingConfig() - ); + final Sequence rows = groupingEngine.process( + query, + cursorFactory, + new IncrementalIndexTimeBoundaryInspector(index), + nonBlockingPool, + null + ); - final List results = rows.toList(); + final List results = rows.toList(); - Assert.assertEquals(1, results.size()); + Assert.assertEquals(1, results.size()); - ResultRow row = results.get(0); - Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); - } + ResultRow row = results.get(0); + Assert.assertArrayEquals(new Object[]{"hi", NullHandling.defaultStringValue(), 1L}, row.getArray()); } @Test @@ -701,6 +688,13 @@ public int hashCode() return super.hashCode(); } + @Override + public boolean equals(Object obj) + { + // Test code, hashcode and equals isn't important + return super.equals(obj); + } + private class DictionaryRaceTestFilterDruidPredicateFactory implements DruidPredicateFactory { @Override diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java index 77e0470c5486..e1a7319cab18 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexIngestionTest.java @@ -25,9 +25,12 @@ import org.apache.druid.guice.BuiltInTypesModule; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.LongMaxAggregator; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.segment.CloserRule; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Rule; @@ -69,22 +72,39 @@ public void testOnHeapIncrementalIndexClose() throws Exception { // Prepare the mocks & set close() call count expectation to 1 Aggregator mockedAggregator = EasyMock.createMock(LongMaxAggregator.class); + EasyMock.expect(mockedAggregator.aggregateWithSize()).andReturn(0L).anyTimes(); mockedAggregator.close(); EasyMock.expectLastCall().times(1); - final IncrementalIndex genericIndex = indexCreator.createIndex( + + EasyMock.replay(mockedAggregator); + + final IncrementalIndex incrementalIndex = indexCreator.createIndex( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) - .withMetrics(new LongMaxAggregatorFactory("max", "max")) + .withMetrics(new LongMaxAggregatorFactory("max", "max") + { + @Override + protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) + { + return mockedAggregator; + } + + @Override + public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) + { + return new AggregatorAndSize(mockedAggregator, Long.BYTES); + } + }) .build() ); // This test is specific to the on-heap index - if (!(genericIndex instanceof OnheapIncrementalIndex)) { + if (!(incrementalIndex instanceof OnheapIncrementalIndex)) { return; } - final OnheapIncrementalIndex index = (OnheapIncrementalIndex) genericIndex; + final OnheapIncrementalIndex index = (OnheapIncrementalIndex) incrementalIndex; index.add(new MapBasedInputRow( 0, @@ -92,11 +112,7 @@ public void testOnHeapIncrementalIndexClose() throws Exception ImmutableMap.of("billy", 1, "max", 1) )); - // override the aggregators with the mocks - index.concurrentGet(0)[0] = mockedAggregator; - // close the indexer and validate the expectations - EasyMock.replay(mockedAggregator); index.close(); EasyMock.verify(mockedAggregator); } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java index 80c8207ed605..f5779bf73629 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexMultiValueSpecTest.java @@ -28,10 +28,7 @@ import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.guice.BuiltInTypesModule; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.CloserRule; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -80,15 +77,11 @@ public void test() throws IndexSizeExceededException new StringDimensionSchema("string3", DimensionSchema.MultiValueHandling.SORTED_SET, true) ) ); - IncrementalIndexSchema schema = new IncrementalIndexSchema( - 0, - new TimestampSpec("ds", "auto", null), - Granularities.ALL, - VirtualColumns.EMPTY, - dimensionsSpec, - new AggregatorFactory[0], - false - ); + IncrementalIndexSchema schema = IncrementalIndexSchema.builder() + .withTimestampSpec(new TimestampSpec("ds", "auto", null)) + .withDimensionsSpec(dimensionsSpec) + .withRollup(false) + .build(); Map map = new HashMap() { @Override diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java index 067c078f5bc3..7a99951c6bd7 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexTest.java @@ -332,7 +332,7 @@ public void sameRow() throws IndexSizeExceededException index.add(row); index.add(row); - Assert.assertEquals("rollup".equals(mode) ? 1 : 3, index.size()); + Assert.assertEquals("rollup".equals(mode) ? 1 : 3, index.numRows()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 1ebd1731c331..64ba2679f04c 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -205,7 +205,7 @@ private SmooshedFileMapper smooshify( globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -244,7 +244,7 @@ public void testBasicFunctionality() throws IOException ); bob.setFileMapper(fileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE); + deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE, null); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); @@ -268,7 +268,7 @@ public void testArrayFunctionality() throws IOException ); bob.setFileMapper(arrayFileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(arrayBaseBuffer, bob, ColumnConfig.SELECTION_SIZE); + deserializer.read(arrayBaseBuffer, bob, ColumnConfig.SELECTION_SIZE, null); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); @@ -291,7 +291,8 @@ public void testConcurrency() throws ExecutionException, InterruptedException bob, ColumnConfig.SELECTION_SIZE, bitmapSerdeFactory, - ByteOrder.nativeOrder() + ByteOrder.nativeOrder(), + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index 33eaa4a0312b..aa42d58710da 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -225,7 +225,7 @@ public void testBasicFunctionality() throws IOException bob.setFileMapper(fileMapper); ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.createDeserializer(NestedDataComplexTypeSerde.TYPE_NAME); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE); + deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE, null); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java index 3f237b9396f1..f483e297be01 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -157,7 +157,7 @@ private SmooshedFileMapper smooshify( globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -192,7 +192,8 @@ public void testBasicFunctionality() bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); try (ScalarDoubleColumn column = (ScalarDoubleColumn) supplier.get()) { smokeTest(supplier, column); @@ -210,7 +211,8 @@ public void testConcurrency() throws ExecutionException, InterruptedException bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java index 5fd8ddd299ce..c8830f3aefd0 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -157,7 +157,7 @@ private SmooshedFileMapper smooshify( globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -192,7 +192,8 @@ public void testBasicFunctionality() bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); try (ScalarLongColumn column = (ScalarLongColumn) supplier.get()) { smokeTest(supplier, column); @@ -210,7 +211,8 @@ public void testConcurrency() throws ExecutionException, InterruptedException bitmapSerdeFactory, baseBuffer, bob, - ColumnConfig.SELECTION_SIZE + ColumnConfig.SELECTION_SIZE, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index ee7ef05149ea..d72970b3b120 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -157,7 +157,7 @@ private SmooshedFileMapper smooshify( globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -191,7 +191,8 @@ public void testBasicFunctionality() throws IOException ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); try (StringUtf8DictionaryEncodedColumn column = (StringUtf8DictionaryEncodedColumn) supplier.get()) { smokeTest(supplier, column); @@ -208,7 +209,8 @@ public void testConcurrency() throws ExecutionException, InterruptedException ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 4a9423b87bc1..6aea2ace2348 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -282,7 +282,7 @@ private SmooshedFileMapper smooshify( globalDictionarySortedCollector.getSortedDoubles(), () -> new AutoTypeColumnMerger.ArrayDictionaryMergingIterator( new Iterable[]{globalDictionarySortedCollector.getSortedArrays()}, - serializer.getGlobalLookup() + serializer.getDictionaryIdLookup() ) ); serializer.open(); @@ -317,7 +317,8 @@ public void testBasicFunctionality() throws IOException ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); try (VariantColumn column = (VariantColumn) supplier.get()) { smokeTest(supplier, column, data, expectedTypes); @@ -335,7 +336,8 @@ public void testConcurrency() throws ExecutionException, InterruptedException ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob + bob, + null ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java index c6e6e9359b4a..30aa6468dd7c 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java @@ -66,7 +66,7 @@ public void testDeserializer() { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); final ColumnCapabilities columnCapabilities = builder.build().getCapabilities(); Assert.assertTrue(Types.is(columnCapabilities, ValueType.DOUBLE)); Assert.assertTrue(columnCapabilities.hasNulls().isTrue()); @@ -82,7 +82,7 @@ public void testDimensionSelector() { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -104,7 +104,7 @@ public void testDimensionVectorSelector() { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -132,7 +132,7 @@ public void testVectorObjectSelector() { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.STRING); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -151,7 +151,7 @@ public void testColumnValueSelector() { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -172,7 +172,7 @@ public void testVectorValueSelector() { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); BaseColumn theColumn = holder.getColumn(); @@ -196,7 +196,7 @@ public void testIndexSupplier() { final NullColumnPartSerde partSerde = new NullColumnPartSerde(10, RoaringBitmapSerdeFactory.getInstance()); final ColumnBuilder builder = new ColumnBuilder().setType(ValueType.DOUBLE); - partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT); + partSerde.getDeserializer().read(EMPTY_BUFFER, builder, ColumnConfig.DEFAULT, null); ColumnHolder holder = builder.build(); Assert.assertNull(holder.getIndexSupplier()); } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java index 632a848830d8..a3060f078a2c 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionSelectorsTest.java @@ -25,7 +25,6 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.common.guava.SettableSupplier; import org.apache.druid.data.input.MapBasedInputRow; -import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; @@ -50,7 +49,6 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexCursorFactory; import org.apache.druid.segment.TestObjectColumnSelector; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; @@ -620,15 +618,10 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept // underlying dimension selector. // This occurred during schemaless ingestion with spare dimension values and no explicit null rows, so the // conditions are replicated by this test. See https://github.com/apache/druid/pull/10248 for details - IncrementalIndexSchema schema = new IncrementalIndexSchema( - 0, - new TimestampSpec("time", "millis", DateTimes.nowUtc()), - Granularities.NONE, - VirtualColumns.EMPTY, - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new CountAggregatorFactory("count")}, - true - ); + IncrementalIndexSchema schema = IncrementalIndexSchema.builder() + .withTimestampSpec(new TimestampSpec("time", "millis", DateTimes.nowUtc())) + .withMetrics(new AggregatorFactory[]{new CountAggregatorFactory("count")}) + .build(); IncrementalIndex index = new OnheapIncrementalIndex.Builder().setMaxRowCount(100).setIndexSchema(schema).build(); index.add( diff --git a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java index 14deba7725af..2e4cf91252fe 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java @@ -53,6 +53,7 @@ public CombinedDataSchema( granularitySpec, transformSpec, null, + null, null ); this.multiValuedDimensions = multiValuedDimensions; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java index bda884018812..76a8c16802cb 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/DataSchema.java @@ -25,12 +25,12 @@ import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.Multiset; import com.google.common.collect.TreeMultiset; import org.apache.druid.common.utils.IdUtils; +import org.apache.druid.data.input.impl.AggregateProjectionSpec; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.data.input.impl.InputRowParser; @@ -64,6 +64,17 @@ public class DataSchema { private static final Logger log = new Logger(DataSchema.class); + + public static Builder builder() + { + return new Builder(); + } + + public static Builder builder(DataSchema schema) + { + return new Builder(schema); + } + private final String dataSource; private final AggregatorFactory[] aggregators; private final GranularitySpec granularitySpec; @@ -77,6 +88,7 @@ public class DataSchema // This is used for backward compatibility private InputRowParser inputRowParser; + private List projections; @JsonCreator public DataSchema( @@ -86,6 +98,7 @@ public DataSchema( @JsonProperty("metricsSpec") AggregatorFactory[] aggregators, @JsonProperty("granularitySpec") GranularitySpec granularitySpec, @JsonProperty("transformSpec") TransformSpec transformSpec, + @JsonProperty("projections") @Nullable List projections, @Deprecated @JsonProperty("parser") @Nullable Map parserMap, @JacksonInject ObjectMapper objectMapper ) @@ -110,6 +123,7 @@ public DataSchema( this.granularitySpec = granularitySpec; } this.transformSpec = transformSpec == null ? TransformSpec.NONE : transformSpec; + this.projections = projections; this.parserMap = parserMap; this.objectMapper = objectMapper; @@ -126,33 +140,6 @@ public DataSchema( } } - @VisibleForTesting - public DataSchema( - String dataSource, - TimestampSpec timestampSpec, - DimensionsSpec dimensionsSpec, - AggregatorFactory[] aggregators, - GranularitySpec granularitySpec, - TransformSpec transformSpec - ) - { - this(dataSource, timestampSpec, dimensionsSpec, aggregators, granularitySpec, transformSpec, null, null); - } - - // old constructor for backward compatibility - @Deprecated - public DataSchema( - String dataSource, - Map parserMap, - AggregatorFactory[] aggregators, - GranularitySpec granularitySpec, - TransformSpec transformSpec, - ObjectMapper objectMapper - ) - { - this(dataSource, null, null, aggregators, granularitySpec, transformSpec, parserMap, objectMapper); - } - private static void validateDatasourceName(String dataSource) { IdUtils.validateId("dataSource", dataSource); @@ -368,6 +355,13 @@ public TransformSpec getTransformSpec() return transformSpec; } + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getProjections() + { + return projections; + } + @Deprecated @JsonProperty("parser") @Nullable @@ -403,44 +397,17 @@ public InputRowParser getParser() public DataSchema withGranularitySpec(GranularitySpec granularitySpec) { - return new DataSchema( - dataSource, - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec, - parserMap, - objectMapper - ); + return builder(this).withGranularity(granularitySpec).build(); } public DataSchema withTransformSpec(TransformSpec transformSpec) { - return new DataSchema( - dataSource, - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec, - parserMap, - objectMapper - ); + return builder(this).withTransform(transformSpec).build(); } public DataSchema withDimensionsSpec(DimensionsSpec dimensionsSpec) { - return new DataSchema( - dataSource, - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec, - parserMap, - objectMapper - ); + return builder(this).withDimensions(dimensionsSpec).build(); } @Override @@ -454,7 +421,121 @@ public String toString() ", parserMap=" + parserMap + ", timestampSpec=" + timestampSpec + ", dimensionsSpec=" + dimensionsSpec + + ", projections=" + projections + ", inputRowParser=" + inputRowParser + '}'; } + + public static class Builder + { + private String dataSource; + private AggregatorFactory[] aggregators; + private GranularitySpec granularitySpec; + private TransformSpec transformSpec; + private Map parserMap; + private ObjectMapper objectMapper; + private TimestampSpec timestampSpec; + private DimensionsSpec dimensionsSpec; + private List projections; + + public Builder() + { + + } + + public Builder(DataSchema schema) + { + this.dataSource = schema.dataSource; + this.timestampSpec = schema.timestampSpec; + this.dimensionsSpec = schema.dimensionsSpec; + this.transformSpec = schema.transformSpec; + this.aggregators = schema.aggregators; + this.projections = schema.projections; + this.granularitySpec = schema.granularitySpec; + this.parserMap = schema.parserMap; + this.objectMapper = schema.objectMapper; + } + + public Builder withDataSource(String dataSource) + { + this.dataSource = dataSource; + return this; + } + + public Builder withTimestamp(TimestampSpec timestampSpec) + { + this.timestampSpec = timestampSpec; + return this; + } + + public Builder withDimensions(DimensionsSpec dimensionsSpec) + { + this.dimensionsSpec = dimensionsSpec; + return this; + } + + public Builder withDimensions(List dimensions) + { + this.dimensionsSpec = DimensionsSpec.builder().setDimensions(dimensions).build(); + return this; + } + + public Builder withDimensions(DimensionSchema... dimensions) + { + return withDimensions(Arrays.asList(dimensions)); + } + + public Builder withAggregators(AggregatorFactory... aggregators) + { + this.aggregators = aggregators; + return this; + } + + public Builder withGranularity(GranularitySpec granularitySpec) + { + this.granularitySpec = granularitySpec; + return this; + } + + public Builder withTransform(TransformSpec transformSpec) + { + this.transformSpec = transformSpec; + return this; + } + + public Builder withProjections(List projections) + { + this.projections = projections; + return this; + } + + @Deprecated + public Builder withObjectMapper(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + return this; + } + + @Deprecated + public Builder withParserMap(Map parserMap) + { + this.parserMap = parserMap; + return this; + } + + public DataSchema build() + { + return new DataSchema( + dataSource, + timestampSpec, + dimensionsSpec, + aggregators, + granularitySpec, + transformSpec, + projections, + parserMap, + objectMapper + ); + } + } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index bb5acec7b8f4..16ac5ebfb969 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -1172,7 +1172,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id try { final long startTime = System.nanoTime(); - int numRows = indexToPersist.getIndex().size(); + int numRows = indexToPersist.getIndex().numRows(); // since the sink may have been persisted before it may have lost its // hydrant count, we remember that value in the sinks' metadata, so we have diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 0d7d01253c84..b1cc63d2d565 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -1647,7 +1647,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdWithShardSpec id try { final long startTime = System.nanoTime(); - int numRows = indexToPersist.getIndex().size(); + int numRows = indexToPersist.getIndex().numRows(); final File persistedFile; final File persistDir = createPersistDirIfNeeded(identifier); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java index 98c5e11fa0b2..702bc3e77adb 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/sink/Sink.java @@ -231,7 +231,7 @@ public FireHydrant swap() public boolean swappable() { synchronized (hydrantLock) { - return writable && currHydrant.getIndex() != null && currHydrant.getIndex().size() != 0; + return writable && currHydrant.getIndex() != null && currHydrant.getIndex().numRows() != 0; } } @@ -286,7 +286,7 @@ public int getNumRowsInMemory() return 0; } - return index.size(); + return index.numRows(); } } @@ -327,6 +327,7 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) .withDimensionsSpec(schema.getDimensionsSpec()) .withMetrics(schema.getAggregators()) .withRollup(schema.getGranularitySpec().isRollup()) + .withProjections(schema.getProjections()) .build(); // Build the incremental-index according to the spec that was chosen by the user @@ -381,7 +382,7 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) } currHydrant = new FireHydrant(newIndex, newCount, getSegment().getId()); if (old != null) { - numRowsExcludingCurrIndex.addAndGet(old.getIndex().size()); + numRowsExcludingCurrIndex.addAndGet(old.getIndex().numRows()); } hydrants.add(currHydrant); } else { @@ -448,7 +449,7 @@ public Iterator iterator() public boolean apply(FireHydrant input) { final IncrementalIndex index = input.getIndex(); - return index == null || index.size() != 0; + return index == null || index.numRows() != 0; } } ); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 90297dd4af9d..94bd77e810df 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -71,6 +71,11 @@ public class DataSchemaTest extends InitializedNullHandlingTest { + private static ArbitraryGranularitySpec ARBITRARY_GRANULARITY = new ArbitraryGranularitySpec( + Granularities.DAY, + ImmutableList.of(Intervals.of("2014/2015")) + ); + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -92,17 +97,16 @@ public void testDefaultExclusions() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableSet.of("__time", "time", "col1", "col2", "metric1", "metric2"), @@ -130,18 +134,16 @@ public void testExplicitInclude() null ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableSet.of("__time", "dimC", "col1", "metric1", "metric2"), @@ -167,22 +169,28 @@ public void testTransformSpec() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - 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 ExpressionTransform("expr", "concat(dimA,dimA)", TestExprMacroTable.INSTANCE) - ) - ), - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parserMap) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withTransform( + new TransformSpec( + new SelectorDimFilter("dimA", "foo", null), + ImmutableList.of( + new ExpressionTransform( + "expr", + "concat(dimA,dimA)", + TestExprMacroTable.INSTANCE + ) + ) + ) + ) + .withObjectMapper(jsonMapper) + .build(); // Test hack that produces a StringInputRowParser. final StringInputRowParser parser = (StringInputRowParser) schema.getParser(); @@ -233,17 +241,16 @@ public void testOverlapMetricNameAndDim() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); expectedException.expect(DruidException.class); expectedException.expectMessage( @@ -256,25 +263,24 @@ public void testOverlapMetricNameAndDim() @Test public void testOverlapTimeAndDimPositionZero() { - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new LongDimensionSchema("__time"), - new StringDimensionSchema("dimA"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimA"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableList.of("__time", "dimA", "dimB"), @@ -290,25 +296,24 @@ public void testOverlapTimeAndDimPositionZeroWrongType() expectedException.expect(DruidException.class); expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new StringDimensionSchema("__time"), - new StringDimensionSchema("dimA"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("__time"), + new StringDimensionSchema("dimA"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); } @Test @@ -321,50 +326,49 @@ public void testOverlapTimeAndDimPositionOne() + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new StringDimensionSchema("dimA"), - new LongDimensionSchema("__time"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dimA"), + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); } @Test public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder() { - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - ImmutableList.of( - new StringDimensionSchema("dimA"), - new LongDimensionSchema("__time"), - new StringDimensionSchema("dimB") - ) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .setForceSegmentSortByTime(false) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - null, - jsonMapper - ); + DataSchema schema = + DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withTimestamp(new TimestampSpec("time", "auto", null)) + .withDimensions( + DimensionsSpec.builder() + .setDimensions( + ImmutableList.of( + new StringDimensionSchema("dimA"), + new LongDimensionSchema("__time"), + new StringDimensionSchema("dimB") + ) + ) + .setDimensionExclusions(ImmutableList.of("dimC")) + .setForceSegmentSortByTime(false) + .build() + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); Assert.assertEquals( ImmutableList.of("dimA", "__time", "dimB"), @@ -402,14 +406,13 @@ public void testOverlapTimeAndDimLegacy() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); + expectedException.expect(DruidException.class); expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); @@ -442,20 +445,19 @@ public void testDuplicateAggregators() + "[metric3] seen in metricsSpec list (2 occurrences)" ); - DataSchema schema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - new DoubleSumAggregatorFactory("metric1", "col3"), - new DoubleSumAggregatorFactory("metric3", "col4"), - new DoubleSumAggregatorFactory("metric3", "col5"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema schema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2"), + new DoubleSumAggregatorFactory("metric1", "col3"), + new DoubleSumAggregatorFactory("metric3", "col4"), + new DoubleSumAggregatorFactory("metric3", "col5") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); } @Test @@ -510,24 +512,20 @@ public void testEmptyDatasource() ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DruidExceptionMatcher - .invalidInput() - .expectMessageIs("Invalid value for field [dataSource]: must not be null") - .assertThrowsAndMatches( - () -> new DataSchema( - "", - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec( - Granularities.DAY, - ImmutableList.of(Intervals.of("2014/2015")) - ), - null, - jsonMapper - )); + DruidExceptionMatcher.ThrowingSupplier thrower = + () -> DataSchema.builder() + .withDataSource("") + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); + DruidExceptionMatcher.invalidInput() + .expectMessageIs("Invalid value for field [dataSource]: must not be null") + .assertThrowsAndMatches(thrower); } @@ -547,14 +545,11 @@ public void testInvalidWhitespaceDatasource() dataSource ); DruidExceptionMatcher.invalidInput().expectMessageIs(msg).assertThrowsAndMatches( - () -> new DataSchema( - dataSource, - Collections.emptyMap(), - null, - null, - null, - jsonMapper - ) + () -> DataSchema.builder() + .withDataSource(dataSource) + .withParserMap(Collections.emptyMap()) + .withObjectMapper(jsonMapper) + .build() ); } } @@ -686,17 +681,16 @@ public void testSerdeWithUpdatedDataSchemaAddedField() throws IOException ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - DataSchema originalSchema = new DataSchema( - IdUtilsTest.VALID_ID_CHARS, - parser, - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("metric1", "col1"), - new DoubleSumAggregatorFactory("metric2", "col2"), - }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - jsonMapper - ); + DataSchema originalSchema = DataSchema.builder() + .withDataSource(IdUtilsTest.VALID_ID_CHARS) + .withParserMap(parser) + .withAggregators( + new DoubleSumAggregatorFactory("metric1", "col1"), + new DoubleSumAggregatorFactory("metric2", "col2") + ) + .withGranularity(ARBITRARY_GRANULARITY) + .withObjectMapper(jsonMapper) + .build(); String serialized = jsonMapper.writeValueAsString(originalSchema); TestModifiedDataSchema deserialized = jsonMapper.readValue(serialized, TestModifiedDataSchema.class); @@ -734,7 +728,7 @@ public void testSerdeWithUpdatedDataSchemaRemovedField() throws IOException new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric2", "col2"), }, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + ARBITRARY_GRANULARITY, null, parser, jsonMapper, @@ -765,10 +759,16 @@ public void testWithDimensionSpec() Map parserMap = Mockito.mock(Map.class); Mockito.when(newDimSpec.withDimensionExclusions(ArgumentMatchers.any(Set.class))).thenReturn(newDimSpec); - DataSchema oldSchema = new DataSchema("dataSource", tsSpec, oldDimSpec, - new AggregatorFactory[]{aggFactory}, gSpec, - transSpec, parserMap, jsonMapper - ); + DataSchema oldSchema = DataSchema.builder() + .withDataSource("dataSource") + .withTimestamp(tsSpec) + .withDimensions(oldDimSpec) + .withAggregators(aggFactory) + .withGranularity(gSpec) + .withTransform(transSpec) + .withParserMap(parserMap) + .withObjectMapper(jsonMapper) + .build(); DataSchema newSchema = oldSchema.withDimensionsSpec(newDimSpec); Assert.assertSame(oldSchema.getDataSource(), newSchema.getDataSource()); Assert.assertSame(oldSchema.getTimestampSpec(), newSchema.getTimestampSpec()); @@ -795,7 +795,7 @@ public void testCombinedDataSchemaSetsMultiValuedColumnsInfo() .setDimensionExclusions(ImmutableList.of("dimC")) .build(), null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), + ARBITRARY_GRANULARITY, null, multiValuedDimensions ); diff --git a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java index b10509036243..326ad6dbed7c 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/TestModifiedDataSchema.java @@ -56,6 +56,7 @@ public TestModifiedDataSchema( aggregators, granularitySpec, transformSpec, + null, parserMap, objectMapper ); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java index 22034aa33aa9..5f43236e075b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorTester.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.IndexIO; @@ -151,19 +150,18 @@ public BatchAppenderatorTester( Map.class ); - schema = new DataSchema( - DATASOURCE, - null, - null, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - parserMap, - objectMapper - ); + schema = DataSchema.builder() + .withDataSource(DATASOURCE) + .withAggregators( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + ) + .withGranularity( + new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null) + ) + .withParserMap(parserMap) + .withObjectMapper(objectMapper) + .build(); tuningConfig = new TestAppenderatorConfig( TuningConfig.DEFAULT_APPENDABLE_INDEX, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index cd990e76f892..29d758aaed02 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -44,7 +44,6 @@ import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.QueryRunnerTestHelper; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; @@ -135,17 +134,16 @@ public StreamAppenderatorTester( ), Map.class ); - schema = new DataSchema( - DATASOURCE, - parserMap, - new AggregatorFactory[]{ - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("met", "met") - }, - new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), - null, - objectMapper - ); + schema = DataSchema.builder() + .withDataSource(DATASOURCE) + .withParserMap(parserMap) + .withAggregators( + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("met", "met") + ) + .withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)) + .withObjectMapper(objectMapper) + .build(); tuningConfig = new TestAppenderatorConfig( TuningConfig.DEFAULT_APPENDABLE_INDEX, maxRowsInMemory, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index 23ac93db0096..21f627baa085 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -98,14 +98,11 @@ public void setup() EasyMock.replay(appenderatorConfig); appenderator = manager.createBatchAppenderatorForTask( "taskId", - new DataSchema( - "myDataSource", - new TimestampSpec("__time", "millis", null), - null, - null, - new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()), - null - ), + DataSchema.builder() + .withDataSource("myDataSource") + .withTimestamp(new TimestampSpec("__time", "millis", null)) + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList())) + .build(), appenderatorConfig, new SegmentGenerationMetrics(), new NoopDataSegmentPusher(), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java index 9d85ec6c8e6b..750ea06c6534 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/sink/SinkTest.java @@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedSegment; @@ -76,14 +75,14 @@ public class SinkTest extends InitializedNullHandlingTest @Test public void testSwap() throws Exception { - final DataSchema schema = new DataSchema( - "test", - new TimestampSpec(null, null, null), - DimensionsSpec.EMPTY, - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null - ); + final DataSchema schema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions(DimensionsSpec.EMPTY) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null)) + .build(); final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); @@ -256,18 +255,17 @@ public void testAcquireSegmentReferences_twoWithOneSwappedToNull() @Test public void testGetSinkSignature() throws IndexSizeExceededException { - final DataSchema schema = new DataSchema( - "test", - new TimestampSpec(null, null, null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new LongDimensionSchema("dimLong") - )), - new AggregatorFactory[]{new CountAggregatorFactory("rows")}, - new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), - null - ); + final DataSchema schema = + DataSchema.builder() + .withDataSource("test") + .withTimestamp(new TimestampSpec(null, null, null)) + .withDimensions( + new StringDimensionSchema("dim1"), + new LongDimensionSchema("dimLong") + ) + .withAggregators(new CountAggregatorFactory("rows")) + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null)) + .build(); final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); diff --git a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java index b617b7e6b877..c0634ed403c0 100644 --- a/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/org/apache/druid/cli/validate/DruidJsonValidatorTest.java @@ -30,7 +30,6 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -133,14 +132,11 @@ public void testTaskValidator() throws Exception null, new TaskResource("rofl", 2), new IndexTask.IndexIngestionSpec( - new DataSchema( - "foo", - null, - new AggregatorFactory[0], - new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), - null, - jsonMapper - ), + DataSchema.builder() + .withDataSource("foo") + .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null)) + .withObjectMapper(jsonMapper) + .build(), new IndexTask.IndexIOConfig( new LocalInputSource(new File("lol"), "rofl"), new JsonInputFormat(null, null, null, null, null),