From 928f929996e5fa77cfa5c659e015d7490dfefdec Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Sun, 30 Aug 2020 15:46:12 +0300 Subject: [PATCH 01/20] Introduce a Configurable Index Type --- .../benchmark/FilterPartitionBenchmark.java | 5 +- .../FilteredAggregatorBenchmark.java | 5 +- .../GroupByTypeInterfaceBenchmark.java | 5 +- .../IncrementalIndexRowTypeBenchmark.java | 5 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 5 +- .../IncrementalIndexReadBenchmark.java | 5 +- .../indexing/IndexIngestionBenchmark.java | 5 +- .../indexing/IndexMergeBenchmark.java | 5 +- .../indexing/IndexPersistBenchmark.java | 5 +- .../benchmark/query/GroupByBenchmark.java | 5 +- .../druid/benchmark/query/ScanBenchmark.java | 5 +- .../benchmark/query/SearchBenchmark.java | 5 +- .../benchmark/query/TimeseriesBenchmark.java | 5 +- .../druid/benchmark/query/TopNBenchmark.java | 5 +- .../timecompare/TimeCompareBenchmark.java | 5 +- docs/ingestion/index.md | 2 + .../DistinctCountGroupByQueryTest.java | 5 +- .../DistinctCountTimeseriesQueryTest.java | 5 +- .../DistinctCountTopNQueryTest.java | 5 +- .../MaterializedViewSupervisorSpec.java | 1 + .../segment/MapVirtualColumnTestBase.java | 5 +- .../kafka/KafkaIndexTaskTuningConfig.java | 4 + .../KafkaSupervisorTuningConfig.java | 8 +- .../indexing/kafka/KafkaIndexTaskTest.java | 1 + .../kafka/KafkaIndexTaskTuningConfigTest.java | 3 + .../kafka/supervisor/KafkaSupervisorTest.java | 5 + ...estModifiedKafkaIndexTaskTuningConfig.java | 3 + .../kinesis/KinesisIndexTaskTuningConfig.java | 4 + .../KinesisSupervisorTuningConfig.java | 5 + .../kinesis/KinesisIndexTaskSerdeTest.java | 1 + .../kinesis/KinesisIndexTaskTest.java | 1 + .../KinesisIndexTaskTuningConfigTest.java | 3 + .../supervisor/KinesisSupervisorTest.java | 3 + ...tModifiedKinesisIndexTaskTuningConfig.java | 4 + .../druid/indexer/HadoopTuningConfig.java | 19 +++ .../druid/indexer/IndexGeneratorJob.java | 7 +- .../indexer/BatchDeltaIngestionTest.java | 1 + .../DetermineHashedPartitionsJobTest.java | 1 + .../indexer/DeterminePartitionsJobTest.java | 1 + .../indexer/HadoopDruidIndexerConfigTest.java | 1 + .../druid/indexer/HadoopTuningConfigTest.java | 1 + .../druid/indexer/IndexGeneratorJobTest.java | 1 + .../apache/druid/indexer/JobHelperTest.java | 1 + .../indexer/path/GranularityPathSpecTest.java | 1 + .../RealtimeAppenderatorTuningConfig.java | 12 ++ .../indexing/common/task/CompactionTask.java | 1 + .../druid/indexing/common/task/IndexTask.java | 17 ++- .../parallel/ParallelIndexSupervisorTask.java | 1 + .../parallel/ParallelIndexTuningConfig.java | 5 + .../overlord/sampler/InputSourceSampler.java | 5 +- .../SeekableStreamIndexTaskTuningConfig.java | 11 ++ ...penderatorDriverRealtimeIndexTaskTest.java | 1 + .../ClientCompactionTaskQuerySerdeTest.java | 1 + .../common/task/CompactionTaskRunTest.java | 1 + .../common/task/CompactionTaskTest.java | 6 + .../common/task/IndexTaskSerdeTest.java | 6 + .../indexing/common/task/IndexTaskTest.java | 4 + .../common/task/RealtimeIndexTaskTest.java | 1 + .../indexing/common/task/TaskSerdeTest.java | 3 + ...stractParallelIndexSupervisorTaskTest.java | 2 + .../ParallelIndexSupervisorTaskKillTest.java | 1 + ...rallelIndexSupervisorTaskResourceTest.java | 1 + .../ParallelIndexSupervisorTaskSerdeTest.java | 1 + .../ParallelIndexSupervisorTaskTest.java | 1 + .../parallel/ParallelIndexTestingFactory.java | 1 + .../ParallelIndexTuningConfigTest.java | 7 + .../SinglePhaseParallelIndexingTest.java | 1 + .../IngestSegmentFirehoseFactoryTest.java | 5 +- ...estSegmentFirehoseFactoryTimelineTest.java | 5 +- .../indexing/overlord/TaskLifecycleTest.java | 5 + .../SeekableStreamSupervisorStateTest.java | 1 + .../druid/jackson/AppendableIndexModule.java | 46 +++++++ .../druid/jackson/DefaultObjectMapper.java | 1 + .../query/groupby/GroupByQueryHelper.java | 31 +++-- .../incremental/AppendableIndexBuilder.java | 128 ++++++++++++++++++ .../incremental/AppendableIndexSpec.java | 27 ++-- .../segment/incremental/IncrementalIndex.java | 126 ----------------- .../incremental/OffheapIncrementalIndex.java | 112 ++++++++++++++- .../incremental/OnheapIncrementalIndex.java | 37 +++++ .../apache/druid/query/DoubleStorageTest.java | 5 +- .../druid/query/MultiValuedDimensionTest.java | 9 +- .../aggregation/AggregationTestHelper.java | 13 +- .../first/StringFirstTimeseriesQueryTest.java | 5 +- .../last/StringLastTimeseriesQueryTest.java | 5 +- .../DataSourceMetadataQueryTest.java | 5 +- ...ByLimitPushDownInsufficientBufferTest.java | 5 +- ...roupByLimitPushDownMultiNodeMergeTest.java | 5 +- .../groupby/GroupByMultiSegmentTest.java | 5 +- .../GroupByQueryRunnerFactoryTest.java | 5 +- .../groupby/NestedQueryPushDownTest.java | 5 +- .../query/metadata/SegmentAnalyzerTest.java | 5 +- .../query/scan/MultiSegmentScanQueryTest.java | 5 +- .../query/search/SearchQueryRunnerTest.java | 5 +- .../TimeBoundaryQueryRunnerTest.java | 5 +- .../TimeseriesQueryRunnerBonusTest.java | 5 +- .../apache/druid/segment/EmptyIndexTest.java | 5 +- .../apache/druid/segment/IndexBuilder.java | 5 +- .../org/apache/druid/segment/IndexIOTest.java | 9 +- .../druid/segment/IndexMergerTestBase.java | 77 +++++------ .../IndexMergerV9CompatibilityTest.java | 5 +- .../IndexMergerV9WithSpatialIndexTest.java | 17 +-- .../druid/segment/SchemalessIndexTest.java | 13 +- .../org/apache/druid/segment/TestIndex.java | 5 +- .../segment/data/IncrementalIndexTest.java | 33 +++-- .../filter/SpatialFilterBonusTest.java | 17 +-- .../segment/filter/SpatialFilterTest.java | 17 +-- .../IncrementalIndexMultiValueSpecTest.java | 4 +- .../IncrementalIndexRowCompTest.java | 4 +- .../IncrementalIndexRowSizeTest.java | 12 +- .../IncrementalIndexStorageAdapterTest.java | 4 +- .../incremental/IncrementalIndexTest.java | 9 +- .../OnheapIncrementalIndexTest.java | 16 +-- .../virtual/ExpressionSelectorsTest.java | 3 +- .../indexing/RealtimeTuningConfig.java | 14 ++ .../druid/segment/indexing/TuningConfig.java | 45 +++++- .../appenderator/AppenderatorConfig.java | 16 +-- .../appenderator/AppenderatorImpl.java | 5 +- .../UnifiedIndexerAppenderatorsManager.java | 7 + .../realtime/plumber/RealtimePlumber.java | 7 +- .../druid/segment/realtime/plumber/Sink.java | 10 +- .../appenderator/AppenderatorPlumberTest.java | 1 + .../appenderator/AppenderatorTester.java | 1 + ...DefaultOfflineAppenderatorFactoryTest.java | 1 + .../firehose/IngestSegmentFirehoseTest.java | 9 +- .../plumber/RealtimePlumberSchoolTest.java | 11 +- .../segment/realtime/plumber/SinkTest.java | 9 +- .../cli/validate/DruidJsonValidatorTest.java | 1 + 127 files changed, 864 insertions(+), 411 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java rename server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java => processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java (53%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index f7a690fbb4af..6908b72909dc 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -70,6 +70,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.joda.time.Interval; @@ -227,10 +228,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java index 560148b41988..c7eec63cb304 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java @@ -69,6 +69,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -225,10 +226,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex(AggregatorFactory[] metrics) { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(metrics) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query, String vectorize) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 1921b4359a32..19fe385a2926 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -70,6 +70,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -413,11 +414,11 @@ public String getFormatString() private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setConcurrentEventAdd(true) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } @TearDown(Level.Trial) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java index cc8d4a35d93e..d228dfeb84a1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java @@ -29,6 +29,7 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Level; @@ -126,11 +127,11 @@ private MapBasedInputRow getStringRow(long timestamp, int dimensionCount) private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(aggs) .setDeserializeComplexMetrics(false) .setMaxRowCount(MAX_ROWS) - .buildOnheap(); + .build(); } @Setup diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java index fabb86ad4ff1..b6078aecb5c4 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -67,6 +67,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -308,10 +309,10 @@ public void setup() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index a49e122b2e7e..9297792b4490 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -123,7 +124,7 @@ public void setup() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -131,7 +132,7 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java index 84483582641c..f206e38d8654 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -104,7 +105,7 @@ public void setup2() private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -112,7 +113,7 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment * 2) - .buildOnheap(); + .build(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java index b677a966a1c2..c683813c0a0c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; @@ -211,7 +212,7 @@ private SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory(SegmentWrit private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -219,6 +220,6 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java index 755947d7cbe5..c23f810594e0 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -34,6 +34,7 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.openjdk.jmh.annotations.Benchmark; @@ -151,7 +152,7 @@ public void teardown() private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -159,7 +160,7 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index f212fe385293..1f1ef83947ed 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -85,6 +85,7 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -576,7 +577,7 @@ public String getFormatString() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(schemaInfo.getDimensionsSpec()) @@ -586,7 +587,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } @TearDown(Level.Trial) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index b5430769ce63..6248de2847a9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -68,6 +68,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -319,10 +320,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java index 680a179fa05c..714f125b96a1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java @@ -75,6 +75,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -386,10 +387,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java index 875192ae4185..f7244cfdcc7d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java @@ -69,6 +69,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -310,10 +311,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java index b290f30de262..f518289eb61a 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -66,6 +66,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -292,10 +293,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java index 406e62725b77..0208fe16f89e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java @@ -73,6 +73,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -404,10 +405,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .build(); } @Benchmark diff --git a/docs/ingestion/index.md b/docs/ingestion/index.md index 0925675f9200..071c8faecbab 100644 --- a/docs/ingestion/index.md +++ b/docs/ingestion/index.md @@ -712,6 +712,7 @@ is: "metricCompression": "lz4", "longEncoding": "longs" }, + "indexType": "onheap", } ``` @@ -722,6 +723,7 @@ is: |maxRowsInMemory|The maximum number of records to store in memory before persisting to disk. Note that this is the number of rows post-rollup, and so it may not be equal to the number of input records. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).|`1000000`| |maxBytesInMemory|The maximum aggregate size of records, in bytes, to store in the JVM heap before persisting. This is based on a rough estimate of memory usage. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).

Setting maxBytesInMemory to -1 disables this check, meaning Druid will rely entirely on maxRowsInMemory to control memory usage. Setting it to zero means the default value will be used (one-sixth of JVM heap size).

Note that the estimate of memory usage is designed to be an overestimate, and can be especially high when using complex ingest-time aggregators, including sketches. If this causes your indexing workloads to persist to disk too often, you can set maxBytesInMemory to -1 and rely on maxRowsInMemory instead.|One-sixth of max JVM heap size| |indexSpec|Tune how data is indexed. See below for more information.|See table below| +|indexType|Specify which index type to use: `onheap`, or `offheap`.|`onheap`| |Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../development/extensions-core/kafka-ingestion.md#tuningconfig), [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md#tuningconfig), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).|| #### `indexSpec` diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java index 6c7db8eb3638..3da07ff62400 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; import org.junit.Before; @@ -78,7 +79,7 @@ public void teardown() throws IOException @Test public void testGroupByWithDistinctCountAgg() throws Exception { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -87,7 +88,7 @@ public void testGroupByWithDistinctCountAgg() throws Exception ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .buildOnheap(); + .build(); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java index 2cc0526480bf..f553bfcd1d63 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java @@ -35,6 +35,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.junit.Test; @@ -50,7 +51,7 @@ public void testTopNWithDistinctCountAgg() throws Exception { TimeseriesQueryEngine engine = new TimeseriesQueryEngine(); - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -58,7 +59,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index 7b14fba07f11..ef1344c72bd0 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -37,6 +37,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.junit.After; @@ -80,7 +81,7 @@ public void testTopNWithDistinctCountAgg() throws Exception { TopNQueryEngine engine = new TopNQueryEngine(pool); - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -88,7 +89,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); String visitor_id = "visitor_id"; String client_type = "client_type"; 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 dd385130c761..5388388498f2 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 @@ -181,6 +181,7 @@ public HadoopIndexTask createTask(Interval interval, String version, List new IncrementalIndex.Builder() + () -> new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .buildOnheap(), + .build(), input, parser ); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java index f23ee7b0fced..8dc9a2f06617 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -33,6 +34,7 @@ public class KafkaIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningCon { @JsonCreator public KafkaIndexTaskTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @@ -55,6 +57,7 @@ public KafkaIndexTaskTuningConfig( ) { super( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, @@ -81,6 +84,7 @@ public KafkaIndexTaskTuningConfig( public KafkaIndexTaskTuningConfig withBasePersistDirectory(File dir) { return new KafkaIndexTaskTuningConfig( + getAppendableIndexSpec(), getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 2dca7f6edfc5..a9e0bfeeb92e 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -23,7 +23,7 @@ import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.segment.indexing.TuningConfigs; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Duration; import org.joda.time.Period; @@ -67,11 +67,13 @@ public static KafkaSupervisorTuningConfig defaultConfig() null, null, null, + null, null ); } public KafkaSupervisorTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, @@ -100,6 +102,7 @@ public KafkaSupervisorTuningConfig( ) { super( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, @@ -193,7 +196,7 @@ public String toString() "maxRowsInMemory=" + getMaxRowsInMemory() + ", maxRowsPerSegment=" + getMaxRowsPerSegment() + ", maxTotalRows=" + getMaxTotalRows() + - ", maxBytesInMemory=" + TuningConfigs.getMaxBytesInMemoryOrDefault(getMaxBytesInMemory()) + + ", maxBytesInMemory=" + getMaxBytesInMemoryOrDefault() + ", intermediatePersistPeriod=" + getIntermediatePersistPeriod() + ", basePersistDirectory=" + getBasePersistDirectory() + ", maxPendingPersists=" + getMaxPendingPersists() + @@ -219,6 +222,7 @@ public String toString() public KafkaIndexTaskTuningConfig convertToTaskTuningConfig() { return new KafkaIndexTaskTuningConfig( + getAppendableIndexSpec(), getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), 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 f4098fc95a22..9b5373522803 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 @@ -2497,6 +2497,7 @@ private KafkaIndexTask createTask( ) throws JsonProcessingException { final KafkaIndexTaskTuningConfig tuningConfig = new KafkaIndexTaskTuningConfig( + null, 1000, null, maxRowsPerSegment, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 3a930dbeeadd..24845dcec155 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -115,6 +115,7 @@ public void testSerdeWithNonDefaults() throws Exception public void testConvert() { KafkaSupervisorTuningConfig original = new KafkaSupervisorTuningConfig( + null, 1, null, 2, @@ -158,6 +159,7 @@ public void testConvert() public void testSerdeWithModifiedTuningConfigAddedField() throws IOException { KafkaIndexTaskTuningConfig base = new KafkaIndexTaskTuningConfig( + null, 1, null, 2, @@ -206,6 +208,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException { TestModifiedKafkaIndexTaskTuningConfig base = new TestModifiedKafkaIndexTaskTuningConfig( + null, 1, null, 2, 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 905abd19e2ac..251d8376a690 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 @@ -270,6 +270,7 @@ public void testCreateBaseTaskContexts() throws JsonProcessingException null, null, null, + null, null ), null @@ -3070,6 +3071,7 @@ public void testIsTaskCurrent() kafkaHost, dataSchema, new KafkaSupervisorTuningConfig( + null, 1000, null, 50000, @@ -3109,6 +3111,7 @@ public void testIsTaskCurrent() DataSchema modifiedDataSchema = getDataSchema("some other datasource"); KafkaSupervisorTuningConfig modifiedTuningConfig = new KafkaSupervisorTuningConfig( + null, 42, // This is different null, 50000, @@ -3404,6 +3407,7 @@ public KafkaIndexTaskClient build( }; final KafkaSupervisorTuningConfig tuningConfig = new KafkaSupervisorTuningConfig( + null, 1000, null, 50000, @@ -3514,6 +3518,7 @@ public KafkaIndexTaskClient build( }; final KafkaSupervisorTuningConfig tuningConfig = new KafkaSupervisorTuningConfig( + null, 1000, null, 50000, diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java index 27e69e8e7e00..06550e159234 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestModifiedKafkaIndexTaskTuningConfig.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.indexing.kafka.KafkaIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -37,6 +38,7 @@ public class TestModifiedKafkaIndexTaskTuningConfig extends KafkaIndexTaskTuning @JsonCreator public TestModifiedKafkaIndexTaskTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @@ -60,6 +62,7 @@ public TestModifiedKafkaIndexTaskTuningConfig( ) { super( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java index f033a6ddf5f8..428f54faefec 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfig.java @@ -25,6 +25,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -50,6 +51,7 @@ public class KinesisIndexTaskTuningConfig extends SeekableStreamIndexTaskTuningC @JsonCreator public KinesisIndexTaskTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, @@ -78,6 +80,7 @@ public KinesisIndexTaskTuningConfig( ) { super( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, @@ -154,6 +157,7 @@ public int getMaxRecordsPerPoll() public KinesisIndexTaskTuningConfig withBasePersistDirectory(File dir) { return new KinesisIndexTaskTuningConfig( + getAppendableIndexSpec(), getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java index 8c1a5fa16701..7cf49a32da8b 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfig.java @@ -23,6 +23,7 @@ import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorTuningConfig; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Duration; import org.joda.time.Period; @@ -75,11 +76,13 @@ public static KinesisSupervisorTuningConfig defaultConfig() null, null, null, + null, null ); } public KinesisSupervisorTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, @@ -115,6 +118,7 @@ public KinesisSupervisorTuningConfig( ) { super( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, @@ -248,6 +252,7 @@ public String toString() public KinesisIndexTaskTuningConfig convertToTaskTuningConfig() { return new KinesisIndexTaskTuningConfig( + getAppendableIndexSpec(), getMaxRowsInMemory(), getMaxBytesInMemory(), getMaxRowsPerSegment(), 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 9ab058c3941e..f6b3582f96b1 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 @@ -72,6 +72,7 @@ public class KinesisIndexTaskSerdeTest null, null, null, + null, null ); private static final KinesisIndexTaskIOConfig IO_CONFIG = new KinesisIndexTaskIOConfig( 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 4aff50b40ba4..8ff5b43ba9b3 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 @@ -2739,6 +2739,7 @@ private KinesisIndexTask createTask( boolean resetOffsetAutomatically = false; int maxRowsInMemory = 1000; final KinesisIndexTaskTuningConfig tuningConfig = new KinesisIndexTaskTuningConfig( + null, maxRowsInMemory, null, maxRowsPerSegment, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 15e56c032dff..149200b59e8b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -136,6 +136,7 @@ public void testSerdeWithNonDefaults() throws Exception public void testSerdeWithModifiedTuningConfigAddedField() throws IOException { KinesisIndexTaskTuningConfig base = new KinesisIndexTaskTuningConfig( + null, 1, 3L, 2, @@ -195,6 +196,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException { KinesisIndexTaskTuningConfig base = new KinesisIndexTaskTuningConfig( + null, 1, 3L, 2, @@ -282,6 +284,7 @@ public void testResetOffsetAndSkipSequenceNotBothTrue() throws Exception public void testConvert() { KinesisSupervisorTuningConfig original = new KinesisSupervisorTuningConfig( + null, 1, (long) 3, 2, 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 135166250d99..3d57a6bbe1b2 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 @@ -167,6 +167,7 @@ public void setupTest() supervisorRecordSupplier = createMock(KinesisRecordSupplier.class); tuningConfig = new KinesisSupervisorTuningConfig( + null, 1000, null, 50000, @@ -3689,6 +3690,7 @@ public void testIsTaskCurrent() DataSchema modifiedDataSchema = getDataSchema("some other datasource"); KinesisSupervisorTuningConfig modifiedTuningConfig = new KinesisSupervisorTuningConfig( + null, 1000, null, 50000, @@ -4741,6 +4743,7 @@ public KinesisIndexTaskClient build( }; final KinesisSupervisorTuningConfig tuningConfig = new KinesisSupervisorTuningConfig( + null, 1000, null, 50000, diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java index e45168db59a7..5b2e2bd2503c 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/test/TestModifiedKinesisIndexTaskTuningConfig.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.indexing.kinesis.KinesisIndexTaskTuningConfig; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -37,6 +38,7 @@ public class TestModifiedKinesisIndexTaskTuningConfig extends KinesisIndexTaskTu @JsonCreator public TestModifiedKinesisIndexTaskTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, @@ -66,6 +68,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( ) { super( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, maxRowsPerSegment, @@ -98,6 +101,7 @@ public TestModifiedKinesisIndexTaskTuningConfig( public TestModifiedKinesisIndexTaskTuningConfig(KinesisIndexTaskTuningConfig base, String extra) { super( + base.getAppendableIndexSpec(), base.getMaxRowsInMemory(), base.getMaxBytesInMemory(), base.getMaxRowsPerSegment(), diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index 2b80d28f101a..29a8b6edf97b 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -29,6 +29,7 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.indexing.TuningConfig; import javax.annotation.Nullable; @@ -56,6 +57,7 @@ public static HadoopTuningConfig makeDefaultTuningConfig() DEFAULT_SHARD_SPECS, DEFAULT_INDEX_SPEC, DEFAULT_INDEX_SPEC, + DEFAULT_APPENDABLE_INDEX, DEFAULT_ROW_FLUSH_BOUNDARY, 0L, false, @@ -83,6 +85,7 @@ public static HadoopTuningConfig makeDefaultTuningConfig() private final Map> shardSpecs; private final IndexSpec indexSpec; private final IndexSpec indexSpecForIntermediatePersists; + private final AppendableIndexSpec appendableIndexSpec; private final int rowFlushBoundary; private final long maxBytesInMemory; private final boolean leaveIntermediate; @@ -108,6 +111,7 @@ public HadoopTuningConfig( final @JsonProperty("shardSpecs") @Nullable Map> shardSpecs, final @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, final @JsonProperty("indexSpecForIntermediatePersists") @Nullable IndexSpec indexSpecForIntermediatePersists, + final @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, final @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, final @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, final @JsonProperty("leaveIntermediate") boolean leaveIntermediate, @@ -140,6 +144,7 @@ public HadoopTuningConfig( this.rowFlushBoundary = maxRowsInMemory == null ? maxRowsInMemoryCOMPAT == null ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; + this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; @@ -188,6 +193,7 @@ public String getVersion() } @JsonProperty + @Override public DimensionBasedPartitionsSpec getPartitionsSpec() { return partitionsSpec; @@ -200,17 +206,26 @@ public Map> getShardSpecs() } @JsonProperty + @Override public IndexSpec getIndexSpec() { return indexSpec; } @JsonProperty + @Override public IndexSpec getIndexSpecForIntermediatePersists() { return indexSpecForIntermediatePersists; } + @JsonProperty + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + @JsonProperty("maxRowsInMemory") public int getRowFlushBoundary() { @@ -218,6 +233,7 @@ public int getRowFlushBoundary() } @JsonProperty + @Override public long getMaxBytesInMemory() { return maxBytesInMemory; @@ -327,6 +343,7 @@ public HadoopTuningConfig withWorkingPath(String path) shardSpecs, indexSpec, indexSpecForIntermediatePersists, + appendableIndexSpec, rowFlushBoundary, maxBytesInMemory, leaveIntermediate, @@ -357,6 +374,7 @@ public HadoopTuningConfig withVersion(String ver) shardSpecs, indexSpec, indexSpecForIntermediatePersists, + appendableIndexSpec, rowFlushBoundary, maxBytesInMemory, leaveIntermediate, @@ -387,6 +405,7 @@ public HadoopTuningConfig withShardSpecs(Map> specs specs, indexSpec, indexSpecForIntermediatePersists, + appendableIndexSpec, rowFlushBoundary, maxBytesInMemory, leaveIntermediate, diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index 91982fd036fd..9ce140ee2a4b 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -50,7 +50,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.indexing.TuningConfigs; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; @@ -302,11 +301,11 @@ private static IncrementalIndex makeIncrementalIndex( .withRollup(config.getSchema().getDataSchema().getGranularitySpec().isRollup()) .build(); - IncrementalIndex newIndex = new IncrementalIndex.Builder() + IncrementalIndex newIndex = tuningConfig.getAppendableIndexSpec().builder() .setIndexSchema(indexSchema) .setMaxRowCount(tuningConfig.getRowFlushBoundary()) - .setMaxBytesInMemory(TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory())) - .buildOnheap(); + .setMaxBytesInMemory(tuningConfig.getMaxBytesInMemoryOrDefault()) + .build(); if (oldDimOrder != null && !indexSchema.getDimensionsSpec().hasCustomDimensions()) { newIndex.loadDimensionIterable(oldDimOrder, oldCapabilities); 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 4b8d72e78847..644ae240262d 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 @@ -465,6 +465,7 @@ private HadoopDruidIndexerConfig makeHadoopDruidIndexerConfig( null, null, null, + null, false, false, false, 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 93eda9434246..a127d519cc2b 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 @@ -219,6 +219,7 @@ public DetermineHashedPartitionsJobTest( null, null, null, + null, false, false, false, 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 6906eaf5611a..047c1049c0ff 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 @@ -328,6 +328,7 @@ public DeterminePartitionsJobTest( null, null, null, + null, false, false, false, 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 c9937e0d98da..9eabd4187a32 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 @@ -241,6 +241,7 @@ HadoopIngestionSpec build() null, null, null, + null, false, false, false, diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java index c1a7e954897d..1397c0f8afe9 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java @@ -44,6 +44,7 @@ public void testSerde() throws Exception null, null, null, + null, 100, null, true, 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 cd93236f1216..8d59554909b6 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 @@ -529,6 +529,7 @@ public void setUp() throws Exception null, null, null, + null, maxRowsInMemory, maxBytesInMemory, true, 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 1cdb9b200281..3188762bc0b0 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 @@ -165,6 +165,7 @@ public void setup() throws Exception null, null, null, + null, false, false, false, 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 4da9ee41b77d..68505be91cf1 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 @@ -63,6 +63,7 @@ public class GranularityPathSpecTest null, null, null, + null, false, false, false, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index eec9b98d4973..56c71fdb07fc 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -26,6 +26,7 @@ import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -53,6 +54,7 @@ private static File createNewBasePersistDirectory() return FileUtils.createTempDir("druid-realtime-persist"); } + private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; private final long maxBytesInMemory; private final DynamicPartitionsSpec partitionsSpec; @@ -74,6 +76,7 @@ private static File createNewBasePersistDirectory() @JsonCreator public RealtimeAppenderatorTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @@ -93,6 +96,7 @@ public RealtimeAppenderatorTuningConfig( @JsonProperty("maxSavedParseExceptions") @Nullable Integer maxSavedParseExceptions ) { + this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; // initializing this to 0, it will be lazily intialized to a value // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) @@ -135,6 +139,13 @@ public RealtimeAppenderatorTuningConfig( : logParseExceptions; } + @Override + @JsonProperty + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + @Override @JsonProperty public int getMaxRowsInMemory() @@ -260,6 +271,7 @@ public int getMaxSavedParseExceptions() public RealtimeAppenderatorTuningConfig withBasePersistDirectory(File dir) { return new RealtimeAppenderatorTuningConfig( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, partitionsSpec.getMaxRowsPerSegment(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index ba2502f197af..afaf0cfb01e9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -219,6 +219,7 @@ static ParallelIndexTuningConfig getTuningConfig(TuningConfig tuningConfig) return new ParallelIndexTuningConfig( null, indexTuningConfig.getMaxRowsPerSegment(), + indexTuningConfig.getAppendableIndexSpec(), indexTuningConfig.getMaxRowsPerSegment(), indexTuningConfig.getMaxBytesInMemory(), indexTuningConfig.getMaxTotalRows(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 16e03f199624..39bb537d4bd6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -74,6 +74,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.BatchIOConfig; @@ -1118,6 +1119,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private static final boolean DEFAULT_REPORT_PARSE_EXCEPTIONS = false; private static final long DEFAULT_PUSH_TIMEOUT = 0; + private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; private final long maxBytesInMemory; @@ -1189,6 +1191,7 @@ private static PartitionsSpec getPartitionsSpec( public IndexTuningConfig( @JsonProperty("targetPartitionSize") @Deprecated @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsPerSegment") @Deprecated @Nullable Integer maxRowsPerSegment, + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxTotalRows") @Deprecated @Nullable Long maxTotalRows, @@ -1211,6 +1214,7 @@ public IndexTuningConfig( ) { this( + appendableIndexSpec, maxRowsInMemory != null ? maxRowsInMemory : rowFlushBoundary_forBackCompatibility, maxBytesInMemory != null ? maxBytesInMemory : 0, getPartitionsSpec( @@ -1242,10 +1246,11 @@ public IndexTuningConfig( private IndexTuningConfig() { - this(null, null, null, null, null, null, null, null, null, null, null, null, null, null); + this(null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); } private IndexTuningConfig( + @Nullable AppendableIndexSpec appendableIndexSpec, @Nullable Integer maxRowsInMemory, @Nullable Long maxBytesInMemory, @Nullable PartitionsSpec partitionsSpec, @@ -1262,6 +1267,7 @@ private IndexTuningConfig( @Nullable Integer maxSavedParseExceptions ) { + this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) @@ -1300,6 +1306,7 @@ private IndexTuningConfig( public IndexTuningConfig withBasePersistDirectory(File dir) { return new IndexTuningConfig( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, partitionsSpec, @@ -1320,6 +1327,7 @@ public IndexTuningConfig withBasePersistDirectory(File dir) public IndexTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) { return new IndexTuningConfig( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, partitionsSpec, @@ -1337,6 +1345,13 @@ public IndexTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec) ); } + @JsonProperty + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + @JsonProperty @Override public int getMaxRowsInMemory() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 4a218a0b26e0..02df185835cd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -924,6 +924,7 @@ private static IndexTuningConfig convertToIndexTuningConfig(ParallelIndexTuningC return new IndexTuningConfig( null, null, + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), tuningConfig.getMaxBytesInMemory(), null, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java index 935eeb48c21e..ca7fdf13ffed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java @@ -28,6 +28,7 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.java.util.common.IAE; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Duration; import org.joda.time.Period; @@ -97,6 +98,7 @@ public static ParallelIndexTuningConfig defaultConfig() null, null, null, + null, null ); } @@ -105,6 +107,7 @@ public static ParallelIndexTuningConfig defaultConfig() public ParallelIndexTuningConfig( @JsonProperty("targetPartitionSize") @Deprecated @Nullable Integer targetPartitionSize, @JsonProperty("maxRowsPerSegment") @Deprecated @Nullable Integer maxRowsPerSegment, + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") @Nullable Long maxBytesInMemory, @JsonProperty("maxTotalRows") @Deprecated @Nullable Long maxTotalRows, @@ -134,6 +137,7 @@ public ParallelIndexTuningConfig( super( targetPartitionSize, maxRowsPerSegment, + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, maxTotalRows, @@ -248,6 +252,7 @@ public ParallelIndexTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpe return new ParallelIndexTuningConfig( null, null, + getAppendableIndexSpec(), getMaxRowsInMemory(), getMaxBytesInMemory(), null, 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 b0f470209023..6cb1e59fd161 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 @@ -43,6 +43,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.DataSchema; import javax.annotation.Nullable; @@ -210,8 +211,8 @@ private IncrementalIndex buildIncrementalIndex(SamplerConfig sampler .withRollup(dataSchema.getGranularitySpec().isRollup()) .build(); - return new IncrementalIndex.Builder().setIndexSchema(schema) + return (OnheapIncrementalIndex) new OnheapIncrementalIndex.Builder().setIndexSchema(schema) .setMaxRowCount(samplerConfig.getNumRows()) - .buildOnheap(); + .build(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index c192703e2d62..d536db84c33b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.indexing.RealtimeTuningConfig; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; @@ -37,6 +38,7 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements TuningConfi private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; private static final boolean DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK = false; + private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; private final long maxBytesInMemory; private final DynamicPartitionsSpec partitionsSpec; @@ -59,6 +61,7 @@ public abstract class SeekableStreamIndexTaskTuningConfig implements TuningConfi private final int maxSavedParseExceptions; public SeekableStreamIndexTaskTuningConfig( + @Nullable AppendableIndexSpec appendableIndexSpec, @Nullable Integer maxRowsInMemory, @Nullable Long maxBytesInMemory, @Nullable Integer maxRowsPerSegment, @@ -84,6 +87,7 @@ public SeekableStreamIndexTaskTuningConfig( // Cannot be a static because default basePersistDirectory is unique per-instance final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); + this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); // initializing this to 0, it will be lazily initialized to a value @@ -130,6 +134,13 @@ public SeekableStreamIndexTaskTuningConfig( : logParseExceptions; } + @Override + @JsonProperty + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + @Override @JsonProperty public int getMaxRowsInMemory() diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index a9d82b169a20..9a8ae20c1ad8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1394,6 +1394,7 @@ private AppenderatorDriverRealtimeIndexTask makeRealtimeTask( null ); RealtimeAppenderatorTuningConfig tuningConfig = new RealtimeAppenderatorTuningConfig( + null, 1000, null, maxRowsPerSegment, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java index 0692c606b639..0daffc66bbe8 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java @@ -202,6 +202,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException .inputSpec(new CompactionIntervalSpec(Intervals.of("2019/2020"), "testSha256OfSortedSegmentIds")) .tuningConfig( new ParallelIndexTuningConfig( + null, null, null, 40000, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index 08cd765a1fb1..1fce094d110f 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -279,6 +279,7 @@ public void testRunWithHashPartitioning() throws Exception null, null, null, + null, new HashedPartitionsSpec(null, 3, null), null, null, 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 c82f1e7ece37..14c0840be4bd 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 @@ -305,6 +305,7 @@ private static ParallelIndexTuningConfig createTuningConfig() return new ParallelIndexTuningConfig( null, null, // null to compute maxRowsPerSegment automatically + null, 500000, 1000000L, null, @@ -438,6 +439,7 @@ public void testSerdeWithOldTuningConfigSuccessfullyDeserializeToNewOne() throws new IndexTuningConfig( null, null, // null to compute maxRowsPerSegment automatically + null, 500000, 1000000L, null, @@ -583,6 +585,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( 100000, null, + null, 500000, 1000000L, null, @@ -648,6 +651,7 @@ public void testCreateIngestionSchemaWithTargetPartitionSize() throws IOExceptio public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException, SegmentLoadingException { final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, null, null, 500000, @@ -715,6 +719,7 @@ public void testCreateIngestionSchemaWithMaxTotalRows() throws IOException, Segm public void testCreateIngestionSchemaWithNumShards() throws IOException, SegmentLoadingException { final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, null, null, 500000, @@ -1103,6 +1108,7 @@ private void assertIngestionSchema( expectedMetricsSpec, expectedSegmentIntervals, new ParallelIndexTuningConfig( + null, null, null, 500000, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java index 23f5748a39a9..bf2e44428f29 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskSerdeTest.java @@ -57,6 +57,7 @@ public static void setup() public void testSerdeTuningConfigWithDynamicPartitionsSpec() throws IOException { final IndexTuningConfig tuningConfig = new IndexTuningConfig( + null, null, null, 100, @@ -90,6 +91,7 @@ public void testSerdeTuningConfigWithDynamicPartitionsSpec() throws IOException public void testSerdeTuningConfigWithHashedPartitionsSpec() throws IOException { final IndexTuningConfig tuningConfig = new IndexTuningConfig( + null, null, null, 100, @@ -125,6 +127,7 @@ public void testSerdeTuningConfigWithDeprecatedDynamicPartitionsSpec() throws IO final IndexTuningConfig tuningConfig = new IndexTuningConfig( null, 1000, + null, 100, 2000L, 3000L, @@ -156,6 +159,7 @@ public void testSerdeTuningConfigWithDeprecatedDynamicPartitionsSpec() throws IO public void testSerdeTuningConfigWithDeprecatedHashedPartitionsSpec() throws IOException { final IndexTuningConfig tuningConfig = new IndexTuningConfig( + null, null, null, 100, @@ -191,6 +195,7 @@ public void testForceGuaranteedRollupWithDynamicPartitionsSpec() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("DynamicPartitionsSpec cannot be used for perfect rollup"); final IndexTuningConfig tuningConfig = new IndexTuningConfig( + null, null, null, 100, @@ -225,6 +230,7 @@ public void testBestEffortRollupWithHashedPartitionsSpec() expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("DynamicPartitionsSpec must be used for best-effort rollup"); final IndexTuningConfig tuningConfig = new IndexTuningConfig( + null, null, null, 100, 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 fdd30962ea0d..be16a953dbdd 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 @@ -1122,6 +1122,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception null, null, null, + null, new HashedPartitionsSpec(2, null, null), INDEX_SPEC, null, @@ -1250,6 +1251,7 @@ public void testMultipleParseExceptionsFailure() throws Exception null, null, null, + null, new DynamicPartitionsSpec(2, null), INDEX_SPEC, null, @@ -1370,6 +1372,7 @@ public void testMultipleParseExceptionsFailureAtDeterminePartitions() throws Exc null, null, null, + null, new HashedPartitionsSpec(2, null, null), INDEX_SPEC, null, @@ -1808,6 +1811,7 @@ static IndexTuningConfig createTuningConfig( return new IndexTuningConfig( null, maxRowsPerSegment, + null, maxRowsInMemory, maxBytesInMemory, maxTotalRows, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index b87dcf7e63f5..2038441acfa7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -830,6 +830,7 @@ private RealtimeIndexTask makeRealtimeTask( null ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( + null, 1000, null, new Period("P1Y"), 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 9dfbd4e66f66..aa3d33bf2506 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 @@ -247,6 +247,7 @@ public void testIndexTaskSerde() throws Exception ), new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true), new IndexTuningConfig( + null, null, null, 10, @@ -327,6 +328,7 @@ public void testIndexTaskwithResourceSerde() throws Exception ), new IndexIOConfig(null, new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true), new IndexTuningConfig( + null, null, null, 10, @@ -396,6 +398,7 @@ public void testRealtimeIndexTaskSerde() throws Exception ), new RealtimeTuningConfig( + null, 1, 10L, new Period("PT10M"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index af221602aa84..ae3f92ceb441 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -157,6 +157,7 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase null, null, null, + null, 2, null, null, @@ -227,6 +228,7 @@ protected ParallelIndexTuningConfig newTuningConfig( null, null, null, + null, new MaxSizeSplitHintSpec(null, 1), partitionsSpec, null, 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 37fb265b647d..fc193b0e1321 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 @@ -178,6 +178,7 @@ private ParallelIndexSupervisorTask newTask( null, null, null, + null, numTotalSubTasks, null, null, 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 0a30bf727070..be6b0a86737e 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 @@ -427,6 +427,7 @@ private TestSupervisorTask newTask( null, null, null, + null, NUM_SUB_TASKS, null, null, 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 e6b144fc0217..a80e3a8452b7 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 @@ -271,6 +271,7 @@ ParallelIndexIngestionSpec build() null, null, null, + null, partitionsSpec, null, null, 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 ef1db8a31468..b043f85ef5a6 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 @@ -193,6 +193,7 @@ public void testFailToConstructWhenBothAppendToExistingAndForceGuaranteedRollupA appendToExisting ); final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, null, null, 10, 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 931fd24aeb13..c2f712320b91 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 @@ -158,6 +158,7 @@ ParallelIndexTuningConfig build() return new ParallelIndexTuningConfig( 1, null, + null, 3, 4L, 5L, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java index 101c97473485..13ba39b93a65 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java @@ -67,6 +67,7 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException { final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, null, null, 10, @@ -109,6 +110,7 @@ public void testSerdeWithMaxNumConcurrentSubTasks() throws IOException { final int maxNumConcurrentSubTasks = 250; final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, null, null, 10, @@ -151,6 +153,7 @@ public void testSerdeWithMaxNumSubTasks() throws IOException { final int maxNumSubTasks = 250; final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, null, null, 10, @@ -195,6 +198,7 @@ public void testSerdeWithMaxNumSubTasksAndMaxNumConcurrentSubTasks() expectedException.expectMessage("Can't use both maxNumSubTasks and maxNumConcurrentSubTasks"); final int maxNumSubTasks = 250; final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + null, null, null, 10, @@ -236,6 +240,7 @@ public void testConstructorWithHashedPartitionsSpecAndNonForceGuaranteedRollupFa expectedException.expectMessage("DynamicPartitionsSpec must be used for best-effort rollup"); final boolean forceGuaranteedRollup = false; new ParallelIndexTuningConfig( + null, null, null, 10, @@ -277,6 +282,7 @@ public void testConstructorWithSingleDimensionPartitionsSpecAndNonForceGuarantee expectedException.expectMessage("DynamicPartitionsSpec must be used for best-effort rollup"); final boolean forceGuaranteedRollup = false; new ParallelIndexTuningConfig( + null, null, null, 10, @@ -318,6 +324,7 @@ public void testConstructorWithDynamicPartitionsSpecAndForceGuaranteedRollupFail expectedException.expectMessage("cannot be used for perfect rollup"); final boolean forceGuaranteedRollup = true; new ParallelIndexTuningConfig( + null, null, null, 10, 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 b7f107d07255..e444f8091839 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 @@ -352,6 +352,7 @@ public void testWith1MaxNumConcurrentSubTasks() null, null, null, + null, 1, null, null, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 6ab41617614c..01a1d2424e0d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -73,6 +73,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory; @@ -196,10 +197,10 @@ public static Collection constructorFeeder() throws IOException new DoubleSumAggregatorFactory(METRIC_FLOAT_NAME, DIM_FLOAT_NAME) ) .build(); - final IncrementalIndex index = new IncrementalIndex.Builder() + final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(MAX_ROWS * MAX_SHARD_NUMBER) - .buildOnheap(); + .build(); for (Integer i = 0; i < MAX_ROWS; ++i) { index.add(ROW_PARSER.parseBatch(buildRow(i.longValue())).get(0)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 06ebc56e9ec7..400f7da3dbbe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; @@ -250,10 +251,10 @@ private static Map persist(File tmpDir, InputRow... rows) .withDimensionsSpec(ROW_PARSER) .withMetrics(new LongSumAggregatorFactory(METRICS[0], METRICS[0])) .build(); - final IncrementalIndex index = new IncrementalIndex.Builder() + final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(rows.length) - .buildOnheap(); + .build(); for (InputRow row : rows) { try { 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 99ab9a480e9e..48f75d7a82c3 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 @@ -746,6 +746,7 @@ public void testIndexTask() throws Exception new IndexTuningConfig( null, 10000, + null, 10, null, null, @@ -826,6 +827,7 @@ public void testIndexTaskFailure() throws Exception new IndexTuningConfig( null, 10000, + null, 10, null, null, @@ -1251,6 +1253,7 @@ public void testResumeTasks() throws Exception new IndexTuningConfig( null, 10000, + null, 10, null, null, @@ -1358,6 +1361,7 @@ public void testUnifiedAppenderatorsManagerCleanup() throws Exception new IndexTuningConfig( null, 10000, + null, 10, null, null, @@ -1468,6 +1472,7 @@ private RealtimeIndexTask newRealtimeIndexTask() // PlumberSchool - Realtime Index Task always uses RealtimePlumber which is hardcoded in RealtimeIndexTask class ); RealtimeTuningConfig realtimeTuningConfig = new RealtimeTuningConfig( + null, 1000, null, new Period("P1Y"), 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 6bfbd7f9983f..2cefebd6a227 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 @@ -898,6 +898,7 @@ public SeekableStreamIndexTaskTuningConfig convertToTaskTuningConfig() null, null, null, + null, null ) { diff --git a/processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java b/processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java new file mode 100644 index 000000000000..3cc95879e95b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java @@ -0,0 +1,46 @@ +/* + * 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.jackson; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.fasterxml.jackson.databind.module.SimpleModule; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.incremental.OffheapIncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; + +public class AppendableIndexModule extends SimpleModule +{ + public AppendableIndexModule() + { + super("AppendableIndexFactories"); + + setMixInAnnotation(AppendableIndexSpec.class, AppendableIndexSpecMixin.class); + } + + @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = OnheapIncrementalIndex.Spec.class) + @JsonSubTypes(value = { + @JsonSubTypes.Type(name = OnheapIncrementalIndex.Spec.TYPE, value = OnheapIncrementalIndex.Spec.class), + @JsonSubTypes.Type(name = OffheapIncrementalIndex.Spec.TYPE, value = OffheapIncrementalIndex.Spec.class) + }) + public interface AppendableIndexSpecMixin + { + } +} diff --git a/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java b/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java index 235d46731d69..71cf30ff733c 100644 --- a/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java +++ b/processing/src/main/java/org/apache/druid/jackson/DefaultObjectMapper.java @@ -49,6 +49,7 @@ public DefaultObjectMapper(JsonFactory factory) registerModule(new AggregatorsModule()); registerModule(new StringComparatorModule()); registerModule(new SegmentizerModule()); + registerModule(new AppendableIndexModule()); configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); configure(MapperFeature.AUTO_DETECT_GETTERS, false); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java index 877fd34ccf75..6e72cb495c83 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java @@ -39,9 +39,12 @@ 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.incremental.AppendableIndexBuilder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OffheapIncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.joda.time.DateTime; import javax.annotation.Nullable; @@ -117,24 +120,24 @@ public String apply(DimensionSpec input) .withMinTimestamp(granTimeStart.getMillis()) .build(); + + AppendableIndexBuilder indexBuilder; + if (query.getContextValue("useOffheap", false)) { - index = new IncrementalIndex.Builder() - .setIndexSchema(indexSchema) - .setDeserializeComplexMetrics(false) - .setConcurrentEventAdd(true) - .setSortFacts(sortResults) - .setMaxRowCount(querySpecificConfig.getMaxResults()) - .buildOffheap(bufferPool); + indexBuilder = new OffheapIncrementalIndex.Builder() + .setBufferPool(bufferPool); } else { - index = new IncrementalIndex.Builder() - .setIndexSchema(indexSchema) - .setDeserializeComplexMetrics(false) - .setConcurrentEventAdd(true) - .setSortFacts(sortResults) - .setMaxRowCount(querySpecificConfig.getMaxResults()) - .buildOnheap(); + indexBuilder = new OnheapIncrementalIndex.Builder(); } + index = indexBuilder + .setIndexSchema(indexSchema) + .setDeserializeComplexMetrics(false) + .setConcurrentEventAdd(true) + .setSortFacts(sortResults) + .setMaxRowCount(querySpecificConfig.getMaxResults()) + .build(); + Accumulator accumulator = new Accumulator() { @Override diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java new file mode 100644 index 000000000000..2cf6ccd2cc80 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java @@ -0,0 +1,128 @@ +/* + * 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.annotations.VisibleForTesting; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.AggregatorFactory; + +import javax.annotation.Nullable; + +public abstract class AppendableIndexBuilder +{ + @Nullable + protected IncrementalIndexSchema incrementalIndexSchema = null; + protected boolean deserializeComplexMetrics = true; + protected boolean concurrentEventAdd = false; + protected boolean sortFacts = true; + protected int maxRowCount = 0; + protected long maxBytesInMemory = 0; + + protected final Logger log = new Logger(this.getClass().getName()); + + public AppendableIndexBuilder setIndexSchema(final IncrementalIndexSchema incrementalIndexSchema) + { + this.incrementalIndexSchema = incrementalIndexSchema; + return this; + } + + /** + * A helper method to set a simple index schema with only metrics and default values for the other parameters. Note + * that this method is normally used for testing and benchmarking; it is unlikely that you would use it in + * production settings. + * + * @param metrics variable array of {@link AggregatorFactory} metrics + * + * @return this + */ + @VisibleForTesting + public AppendableIndexBuilder setSimpleTestingIndexSchema(final AggregatorFactory... metrics) + { + return setSimpleTestingIndexSchema(null, metrics); + } + + + /** + * A helper method to set a simple index schema with controllable metrics and rollup, and default values for the + * other parameters. Note that this method is normally used for testing and benchmarking; it is unlikely that you + * would use it in production settings. + * + * @param metrics variable array of {@link AggregatorFactory} metrics + * + * @return this + */ + @VisibleForTesting + public AppendableIndexBuilder setSimpleTestingIndexSchema(@Nullable Boolean rollup, final AggregatorFactory... metrics) + { + IncrementalIndexSchema.Builder builder = new IncrementalIndexSchema.Builder().withMetrics(metrics); + this.incrementalIndexSchema = rollup != null ? builder.withRollup(rollup).build() : builder.build(); + return this; + } + + public AppendableIndexBuilder setDeserializeComplexMetrics(final boolean deserializeComplexMetrics) + { + this.deserializeComplexMetrics = deserializeComplexMetrics; + return this; + } + + public AppendableIndexBuilder setConcurrentEventAdd(final boolean concurrentEventAdd) + { + this.concurrentEventAdd = concurrentEventAdd; + return this; + } + + public AppendableIndexBuilder setSortFacts(final boolean sortFacts) + { + this.sortFacts = sortFacts; + return this; + } + + public AppendableIndexBuilder setMaxRowCount(final int maxRowCount) + { + this.maxRowCount = maxRowCount; + return this; + } + + public AppendableIndexBuilder setMaxBytesInMemory(final long maxBytesInMemory) + { + this.maxBytesInMemory = maxBytesInMemory; + return this; + } + + public void validate() + { + if (maxRowCount <= 0) { + throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); + } + + if (incrementalIndexSchema == null) { + throw new IllegalArgumentException("incrementIndexSchema cannot be null"); + } + } + + public final IncrementalIndex build() + { + log.info("Building appendable index."); + validate(); + return buildInner(); + } + + protected abstract IncrementalIndex buildInner(); +} diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java similarity index 53% rename from server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java rename to processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java index 9c470ab677d0..f9847ede47cd 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfigs.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java @@ -17,25 +17,16 @@ * under the License. */ -package org.apache.druid.segment.indexing; +package org.apache.druid.segment.incremental; -public class TuningConfigs +import org.apache.druid.guice.annotations.ExtensionPoint; + +@ExtensionPoint +public interface AppendableIndexSpec { - private TuningConfigs() - { - } + // Returns a builder of the appendable index. + AppendableIndexBuilder builder(); - public static long getMaxBytesInMemoryOrDefault(final long maxBytesInMemory) - { - // In the main tuningConfig class constructor, we set the maxBytes to 0 if null to avoid setting - // maxBytes to max jvm memory of the process that starts first. Instead we set the default based on - // the actual task node's jvm memory. - long newMaxBytesInMemory = maxBytesInMemory; - if (maxBytesInMemory == 0) { - newMaxBytesInMemory = TuningConfig.DEFAULT_MAX_BYTES_IN_MEMORY; - } else if (maxBytesInMemory < 0) { - newMaxBytesInMemory = Long.MAX_VALUE; - } - return newMaxBytesInMemory; - } + // Returns the default max bytes in memory for this index. + long getDefaultMaxBytesInMemory(); } 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 a718b53646d9..ccf0ea65dbdb 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 @@ -31,7 +31,6 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.errorprone.annotations.concurrent.GuardedBy; -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.MapBasedRow; @@ -79,7 +78,6 @@ import javax.annotation.Nullable; import java.io.Closeable; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.Comparator; @@ -88,7 +86,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; @@ -319,129 +316,6 @@ protected IncrementalIndex( } } - public static class Builder - { - @Nullable - private IncrementalIndexSchema incrementalIndexSchema; - private boolean deserializeComplexMetrics; - private boolean concurrentEventAdd; - private boolean sortFacts; - private int maxRowCount; - private long maxBytesInMemory; - - public Builder() - { - incrementalIndexSchema = null; - deserializeComplexMetrics = true; - concurrentEventAdd = false; - sortFacts = true; - maxRowCount = 0; - maxBytesInMemory = 0; - } - - public Builder setIndexSchema(final IncrementalIndexSchema incrementalIndexSchema) - { - this.incrementalIndexSchema = incrementalIndexSchema; - return this; - } - - /** - * A helper method to set a simple index schema with only metrics and default values for the other parameters. Note - * that this method is normally used for testing and benchmarking; it is unlikely that you would use it in - * production settings. - * - * @param metrics variable array of {@link AggregatorFactory} metrics - * - * @return this - */ - @VisibleForTesting - public Builder setSimpleTestingIndexSchema(final AggregatorFactory... metrics) - { - return setSimpleTestingIndexSchema(null, metrics); - } - - - /** - * A helper method to set a simple index schema with controllable metrics and rollup, and default values for the - * other parameters. Note that this method is normally used for testing and benchmarking; it is unlikely that you - * would use it in production settings. - * - * @param metrics variable array of {@link AggregatorFactory} metrics - * - * @return this - */ - @VisibleForTesting - public Builder setSimpleTestingIndexSchema(@Nullable Boolean rollup, final AggregatorFactory... metrics) - { - IncrementalIndexSchema.Builder builder = new IncrementalIndexSchema.Builder().withMetrics(metrics); - this.incrementalIndexSchema = rollup != null ? builder.withRollup(rollup).build() : builder.build(); - return this; - } - - public Builder setDeserializeComplexMetrics(final boolean deserializeComplexMetrics) - { - this.deserializeComplexMetrics = deserializeComplexMetrics; - return this; - } - - public Builder setConcurrentEventAdd(final boolean concurrentEventAdd) - { - this.concurrentEventAdd = concurrentEventAdd; - return this; - } - - public Builder setSortFacts(final boolean sortFacts) - { - this.sortFacts = sortFacts; - return this; - } - - public Builder setMaxRowCount(final int maxRowCount) - { - this.maxRowCount = maxRowCount; - return this; - } - - //maxBytesInMemory only applies to OnHeapIncrementalIndex - public Builder setMaxBytesInMemory(final long maxBytesInMemory) - { - this.maxBytesInMemory = maxBytesInMemory; - return this; - } - - public OnheapIncrementalIndex buildOnheap() - { - if (maxRowCount <= 0) { - throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); - } - - return new OnheapIncrementalIndex( - Objects.requireNonNull(incrementalIndexSchema, "incrementIndexSchema is null"), - deserializeComplexMetrics, - concurrentEventAdd, - sortFacts, - maxRowCount, - maxBytesInMemory - ); - } - - public IncrementalIndex buildOffheap(final NonBlockingPool bufferPool) - { - if (maxRowCount <= 0) { - throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); - } - - return new OffheapIncrementalIndex( - Objects.requireNonNull(incrementalIndexSchema, "incrementalIndexSchema is null"), - deserializeComplexMetrics, - concurrentEventAdd, - sortFacts, - maxRowCount, - Objects.requireNonNull(bufferPool, "bufferPool is null") - ); - } - } - public abstract FactsHolder getFacts(); diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java index 97da9945a06e..406005f84676 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java @@ -19,9 +19,12 @@ package org.apache.druid.segment.incremental; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.collections.StupidPool; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -32,6 +35,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.io.IOException; @@ -40,6 +44,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; /** @@ -115,6 +120,8 @@ protected BufferAggregator[] initAggs( selectors = new HashMap<>(); aggOffsetInBuffer = new int[metrics.length]; + int aggsCurOffsetInBuffer = 0; + for (int i = 0; i < metrics.length; i++) { AggregatorFactory agg = metrics[i]; @@ -129,15 +136,11 @@ protected BufferAggregator[] initAggs( new OnheapIncrementalIndex.CachingColumnSelectorFactory(columnSelectorFactory, concurrentEventAdd) ); - if (i == 0) { - aggOffsetInBuffer[i] = 0; - } else { - aggOffsetInBuffer[i] = aggOffsetInBuffer[i - 1] + metrics[i - 1].getMaxIntermediateSizeWithNulls(); - } + aggOffsetInBuffer[i] = aggsCurOffsetInBuffer; + aggsCurOffsetInBuffer += agg.getMaxIntermediateSizeWithNulls(); } - aggsTotalSize = aggOffsetInBuffer[metrics.length - 1] + metrics[metrics.length - - 1].getMaxIntermediateSizeWithNulls(); + aggsTotalSize = aggsCurOffsetInBuffer; return new BufferAggregator[metrics.length]; } @@ -346,4 +349,99 @@ public void close() } aggBuffers.clear(); } + + public static class Builder extends AppendableIndexBuilder + { + @Nullable + NonBlockingPool bufferPool = null; + + public Builder setBufferPool(final NonBlockingPool bufferPool) + { + this.bufferPool = bufferPool; + return this; + } + + @Override + public void validate() + { + super.validate(); + if (bufferPool == null) { + throw new IllegalArgumentException("bufferPool cannot be null"); + } + } + + @Override + protected OffheapIncrementalIndex buildInner() + { + return new OffheapIncrementalIndex( + Objects.requireNonNull(incrementalIndexSchema, "incrementalIndexSchema is null"), + deserializeComplexMetrics, + concurrentEventAdd, + sortFacts, + maxRowCount, + Objects.requireNonNull(bufferPool, "bufferPool is null") + ); + } + } + + public static class Spec implements AppendableIndexSpec, Supplier + { + public static final String TYPE = "offheap"; + static final int DEFAULT_BUFFER_SIZE = 1 << 23; + static final int DEFAULT_CACHE_SIZE = 1 << 30; + + final int bufferSize; + final int cacheSize; + final NonBlockingPool bufferPool; + + @JsonCreator + public Spec( + final @JsonProperty("bufferSize") @Nullable Integer bufferSize, + final @JsonProperty("cacheSize") @Nullable Integer cacheSize + ) + { + this.bufferSize = bufferSize != null && bufferSize > 0 ? bufferSize : DEFAULT_BUFFER_SIZE; + this.cacheSize = cacheSize != null && cacheSize > this.bufferSize ? cacheSize : DEFAULT_CACHE_SIZE; + this.bufferPool = new StupidPool<>( + "Offheap incremental-index buffer pool", + this, + 0, + this.cacheSize / this.bufferSize + ); + } + + @JsonProperty + public int getBufferSize() + { + return bufferSize; + } + + @JsonProperty + public int getCacheSize() + { + return cacheSize; + } + + @Override + public ByteBuffer get() + { + return ByteBuffer.allocateDirect(bufferSize); + } + + @Override + public AppendableIndexBuilder builder() + { + return new Builder().setBufferPool(bufferPool); + } + + @Override + public long getDefaultMaxBytesInMemory() + { + // In the realtime node, the entire JVM's direct memory is utilized for ingestion and persist operations. + // But maxBytesInMemory only refers to the active index size and not to the index being flushed to disk and the + // persist buffer. + // To account for that, we set default to 1/2 of the max jvm's direct memory. + return JvmUtils.getRuntimeInfo().getDirectMemorySizeBytes() / 2; + } + } } 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 9bbdf60dbf79..889307993081 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 @@ -32,6 +32,7 @@ import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.io.IOException; @@ -40,6 +41,7 @@ import java.util.HashMap; 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; @@ -434,4 +436,39 @@ public ColumnCapabilities getColumnCapabilities(String columnName) } } + public static class Builder extends AppendableIndexBuilder + { + @Override + protected OnheapIncrementalIndex buildInner() + { + return new OnheapIncrementalIndex( + Objects.requireNonNull(incrementalIndexSchema, "incrementIndexSchema is null"), + deserializeComplexMetrics, + concurrentEventAdd, + sortFacts, + maxRowCount, + maxBytesInMemory + ); + } + } + + public static class Spec implements AppendableIndexSpec + { + public static final String TYPE = "onheap"; + + @Override + public AppendableIndexBuilder builder() + { + return new Builder(); + } + + @Override + public long getDefaultMaxBytesInMemory() + { + // We initially estimated this to be 1/3(max jvm memory), but bytesCurrentlyInMemory only + // tracks active index and not the index being flushed to disk, to account for that + // we halved default to 1/6(max jvm memory) + return JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() / 6; + } + } } diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index d6d27b88332a..ee58422828e1 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -55,6 +55,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -321,10 +322,10 @@ private static QueryableIndex buildIndex(String storeDoubleAsFloat) throws IOExc ) .build(); - final IncrementalIndex index = new IncrementalIndex.Builder() + final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(MAX_ROWS) - .buildOnheap(); + .build(); getStreamOfEvents().forEach(o -> { diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index 33e16db52430..be6b2072fb1a 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -63,6 +63,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -140,10 +141,10 @@ public MultiValuedDimensionTest(final GroupByQueryConfig config, SegmentWriteOut @Before public void setup() throws Exception { - incrementalIndex = new IncrementalIndex.Builder() + incrementalIndex = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(5000) - .buildOnheap(); + .build(); StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( @@ -183,10 +184,10 @@ public void setup() throws Exception "UTF-8" ); - incrementalIndexNullSampler = new IncrementalIndex.Builder() + incrementalIndexNullSampler = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(5000) - .buildOnheap(); + .build(); String[] rowsNullSampler = new String[]{ "{\"time\":\"2011-01-13T00:00:00.000Z\",\"product\":\"product_1\",\"tags\":[],\"othertags\":[\"u1\", \"u2\"]}", diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index 379b04a64a4b..d9b01c28a415 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -75,6 +75,7 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.junit.rules.TemporaryFolder; @@ -429,7 +430,7 @@ public void createIndex( List toMerge = new ArrayList<>(); try { - index = new IncrementalIndex.Builder() + index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -441,7 +442,7 @@ public void createIndex( ) .setDeserializeComplexMetrics(deserializeComplexMetrics) .setMaxRowCount(maxRowCount) - .buildOnheap(); + .build(); while (rows.hasNext()) { Object row = rows.next(); @@ -450,7 +451,7 @@ public void createIndex( toMerge.add(tmp); indexMerger.persist(index, tmp, new IndexSpec(), null); index.close(); - index = new IncrementalIndex.Builder() + index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -462,7 +463,7 @@ public void createIndex( ) .setDeserializeComplexMetrics(deserializeComplexMetrics) .setMaxRowCount(maxRowCount) - .buildOnheap(); + .build(); } if (row instanceof String && parser instanceof StringInputRowParser) { //Note: this is required because StringInputRowParser is InputRowParser as opposed to @@ -519,7 +520,7 @@ public static IncrementalIndex createIncrementalIndex( boolean rollup ) throws Exception { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -530,7 +531,7 @@ public static IncrementalIndex createIncrementalIndex( ) .setDeserializeComplexMetrics(deserializeComplexMetrics) .setMaxRowCount(maxRowCount) - .buildOnheap(); + .build(); while (rows.hasNext()) { Object row = rows.next(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index 445c6c9af419..3f833086d81d 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; @@ -69,7 +70,7 @@ public void setUp() throws IndexSizeExceededException final SerializablePairLongStringSerde serde = new SerializablePairLongStringSerde(); ComplexMetrics.registerSerde(serde.getTypeName(), serde); - incrementalIndex = new IncrementalIndex.Builder() + incrementalIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -78,7 +79,7 @@ public void setUp() throws IndexSizeExceededException .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); incrementalIndex.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index 7765ec4335a8..ae79cb546b5a 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.junit.Before; @@ -68,7 +69,7 @@ public void setUp() throws IndexSizeExceededException final SerializablePairLongStringSerde serde = new SerializablePairLongStringSerde(); ComplexMetrics.registerSerde(serde.getTypeName(), serde); - incrementalIndex = new IncrementalIndex.Builder() + incrementalIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -77,7 +78,7 @@ public void setUp() throws IndexSizeExceededException .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); incrementalIndex.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java index d22ba99efe94..5ac6b585220c 100644 --- a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java @@ -41,6 +41,7 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.LogicalSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; @@ -114,10 +115,10 @@ public void testContextSerde() throws Exception @Test public void testMaxIngestedEventTime() throws Exception { - final IncrementalIndex rtIndex = new IncrementalIndex.Builder() + final IncrementalIndex rtIndex = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .buildOnheap(); + .build(); final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner( new DataSourceMetadataQueryRunnerFactory( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index a2e994ea0abf..f22701d8f69f 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -73,6 +73,7 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -133,7 +134,7 @@ public int columnCacheSizeBytes() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -149,7 +150,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 357a1e2bcd2a..3c62e11ea1a9 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -79,6 +79,7 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -140,7 +141,7 @@ public int columnCacheSizeBytes() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -156,7 +157,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index 4f9b1ffd4782..7b73cc149b59 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -70,6 +70,7 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.junit.After; @@ -127,7 +128,7 @@ public int columnCacheSizeBytes() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -143,7 +144,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index 5c62fed9847c..6f84d576599a 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.junit.After; import org.junit.Before; @@ -138,11 +139,11 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) private Segment createSegment() throws Exception { - IncrementalIndex incrementalIndex = new IncrementalIndex.Builder() + IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setConcurrentEventAdd(true) .setMaxRowCount(5000) - .buildOnheap(); + .build(); StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index 9d793c73fd81..8a97e891f3f5 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -77,6 +77,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.junit.After; @@ -125,7 +126,7 @@ public class NestedQueryPushDownTest private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -142,7 +143,7 @@ private IncrementalIndex makeIncIndex() ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index 6f56e0df94b1..11e00f76a505 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; @@ -275,10 +276,10 @@ public void testAnalyzingSegmentWithNonExistentAggregator() throws IOException .withRollup(true) .build(); - final IncrementalIndex retVal = new IncrementalIndex.Builder() + final IncrementalIndex retVal = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .buildOnheap(); + .build(); IncrementalIndex incrementalIndex = TestIndex.loadIncrementalIndex(retVal, source); QueryableIndex queryableIndex = TestIndex.persistRealtimeAndLoadMMapped(incrementalIndex); SegmentAnalyzer analyzer = new SegmentAnalyzer(EnumSet.of(SegmentMetadataQuery.AnalysisType.SIZE)); diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java index a579df09adf5..93f46d313958 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; @@ -151,10 +152,10 @@ private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount) .withQueryGranularity(Granularities.HOUR) .withMetrics(TestIndex.METRIC_AGGS) .build(); - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(maxRowCount) - .buildOnheap(); + .build(); } @AfterClass diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java index aebcf257b3e5..16a3590d9974 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java @@ -59,6 +59,7 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.junit.Assert; @@ -719,14 +720,14 @@ public void testSearchOnFloatColumnWithExFn() @Test public void testSearchWithNullValueInDimension() throws Exception { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) .build() ) .setMaxRowCount(10) - .buildOnheap(); + .build(); index.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index ecbe8af23768..6834cec5e178 100644 --- a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -41,6 +41,7 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -117,10 +118,10 @@ private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount) .withQueryGranularity(Granularities.HOUR) .withMetrics(TestIndex.METRIC_AGGS) .build(); - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(maxRowCount) - .buildOnheap(); + .build(); } private static SegmentId makeIdentifier(IncrementalIndex index, String version) 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 a0d33e10ea42..8413c7ea0ed4 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 @@ -38,6 +38,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.junit.Assert; import org.junit.Test; @@ -66,14 +67,14 @@ public TimeseriesQueryRunnerBonusTest(boolean descending) @Test public void testOneRowAtATime() throws Exception { - final IncrementalIndex oneRowIndex = new IncrementalIndex.Builder() + final IncrementalIndex oneRowIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2012-01-01T00:00:00Z").getMillis()) .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); List> results; diff --git a/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java b/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java index e9e8a17e6b21..306cb921b300 100644 --- a/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAdapter; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; @@ -72,10 +73,10 @@ public void testEmptyIndex() throws Exception } try { - IncrementalIndex emptyIndex = new IncrementalIndex.Builder() + IncrementalIndex emptyIndex = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(/* empty */) .setMaxRowCount(1000) - .buildOnheap(); + .build(); IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter( Intervals.of("2012-08-01/P3D"), diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 5813f6d298ec..be2f36399660 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -32,6 +32,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -225,10 +226,10 @@ private static IncrementalIndex buildIncrementalIndexWithRows( ) { Preconditions.checkNotNull(schema, "schema"); - final IncrementalIndex incrementalIndex = new IncrementalIndex.Builder() + final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(maxRows) - .buildOnheap(); + .build(); for (InputRow row : rows) { try { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java index 2fb6bc39eaa4..5066bef4ac54 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java @@ -40,6 +40,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; import org.junit.Assert; @@ -253,7 +254,7 @@ public IndexIOTest( this.exception = exception; } - final IncrementalIndex incrementalIndex1 = new IncrementalIndex.Builder() + final IncrementalIndex incrementalIndex1 = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DEFAULT_INTERVAL.getStart().getMillis()) @@ -268,9 +269,9 @@ public IndexIOTest( .build() ) .setMaxRowCount(1000000) - .buildOnheap(); + .build(); - final IncrementalIndex incrementalIndex2 = new IncrementalIndex.Builder() + final IncrementalIndex incrementalIndex2 = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DEFAULT_INTERVAL.getStart().getMillis()) @@ -285,7 +286,7 @@ public IndexIOTest( .build() ) .setMaxRowCount(1000000) - .buildOnheap(); + .build(); IndexableAdapter adapter1; IndexableAdapter adapter2; 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 86e8503f997f..40516f952396 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -57,6 +57,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -270,10 +271,10 @@ public void testPersistMerge() throws Exception IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); IncrementalIndexTest.populateIndex(timestamp, toPersist1); - IncrementalIndex toPersist2 = new IncrementalIndex.Builder() + IncrementalIndex toPersist2 = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersist2.add( new MapBasedInputRow( @@ -343,15 +344,15 @@ public void testPersistMerge() throws Exception @Test public void testPersistEmptyColumn() throws Exception { - final IncrementalIndex toPersist1 = new IncrementalIndex.Builder() + final IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(/* empty */) .setMaxRowCount(10) - .buildOnheap(); + .build(); - final IncrementalIndex toPersist2 = new IncrementalIndex.Builder() + final IncrementalIndex toPersist2 = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(/* empty */) .setMaxRowCount(10) - .buildOnheap(); + .build(); final File tmpDir1 = temporaryFolder.newFolder(); final File tmpDir2 = temporaryFolder.newFolder(); @@ -834,18 +835,18 @@ public void testMergeWithDimensionsList() throws Exception .build(); - IncrementalIndex toPersist1 = new IncrementalIndex.Builder() + IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .buildOnheap(); - IncrementalIndex toPersist2 = new IncrementalIndex.Builder() + .build(); + IncrementalIndex toPersist2 = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .buildOnheap(); - IncrementalIndex toPersist3 = new IncrementalIndex.Builder() + .build(); + IncrementalIndex toPersist3 = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); addDimValuesToIndex(toPersist1, "dimA", Arrays.asList("1", "2")); addDimValuesToIndex(toPersist2, "dimA", Arrays.asList("1", "2")); @@ -1019,10 +1020,10 @@ public void testJointDimMerge() throws Exception for (IncrementalIndexSchema indexSchema : Arrays.asList(rollupIndexSchema, noRollupIndexSchema)) { - IncrementalIndex toPersistA = new IncrementalIndex.Builder() + IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersistA.add( new MapBasedInputRow( @@ -1039,10 +1040,10 @@ public void testJointDimMerge() throws Exception ) ); - IncrementalIndex toPersistB = new IncrementalIndex.Builder() + IncrementalIndex toPersistB = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersistB.add( new MapBasedInputRow( @@ -1184,10 +1185,10 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception .withMetrics(new CountAggregatorFactory("count")) .withRollup(false) .build(); - IncrementalIndex toPersistA = new IncrementalIndex.Builder() + IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersistA.add( new MapBasedInputRow( @@ -1208,10 +1209,10 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception ) ); - IncrementalIndex toPersistB = new IncrementalIndex.Builder() + IncrementalIndex toPersistB = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersistB.add( new MapBasedInputRow( @@ -1320,10 +1321,10 @@ public void testMergeWithSupersetOrdering() throws Exception IncrementalIndex toPersistBA = getSingleDimIndex("dimB", Arrays.asList("1", "2", "3")); addDimValuesToIndex(toPersistBA, "dimA", Arrays.asList("1", "2")); - IncrementalIndex toPersistBA2 = new IncrementalIndex.Builder() + IncrementalIndex toPersistBA2 = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersistBA2.add( new MapBasedInputRow( @@ -1867,10 +1868,10 @@ private IncrementalIndex getIndexWithDimsFromSchemata(List dims .withMetrics(new CountAggregatorFactory("count")) .build(); - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } @@ -1914,10 +1915,10 @@ public void testPersistNullColumnSkipping() throws Exception private IncrementalIndex getIndexD3() throws Exception { - IncrementalIndex toPersist1 = new IncrementalIndex.Builder() + IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersist1.add( new MapBasedInputRow( @@ -1948,10 +1949,10 @@ private IncrementalIndex getIndexD3() throws Exception private IncrementalIndex getSingleDimIndex(String dimName, List values) throws Exception { - IncrementalIndex toPersist1 = new IncrementalIndex.Builder() + IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .buildOnheap(); + .build(); addDimValuesToIndex(toPersist1, dimName, values); return toPersist1; @@ -1971,10 +1972,10 @@ private IncrementalIndex getIndexWithDims(List dims) .withMetrics(new CountAggregatorFactory("count")) .build(); - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } private AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] aggregators) @@ -2207,10 +2208,10 @@ public void testMultivalDim_mergeAcrossSegments_rollupWorks() throws Exception .withRollup(true) .build(); - IncrementalIndex toPersistA = new IncrementalIndex.Builder() + IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); Map event1 = new HashMap<>(); event1.put("dimA", "leek"); @@ -2225,10 +2226,10 @@ public void testMultivalDim_mergeAcrossSegments_rollupWorks() throws Exception toPersistA.add(new MapBasedInputRow(1, dims, event1)); toPersistA.add(new MapBasedInputRow(1, dims, event2)); - IncrementalIndex toPersistB = new IncrementalIndex.Builder() + IncrementalIndex toPersistB = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); Map event3 = new HashMap<>(); event3.put("dimA", "leek"); @@ -2453,10 +2454,10 @@ public void testMultivalDim_persistAndMerge_dimensionValueOrderingRules() throws multivalEvent9 ); - IncrementalIndex toPersistA = new IncrementalIndex.Builder() + IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); for (Map event : events) { toPersistA.add(new MapBasedInputRow(1, dims, event)); @@ -2469,10 +2470,10 @@ public void testMultivalDim_persistAndMerge_dimensionValueOrderingRules() throws List singleEventIndexes = new ArrayList<>(); for (Map event : events) { - IncrementalIndex toPersist = new IncrementalIndex.Builder() + IncrementalIndex toPersist = new OnheapIncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .buildOnheap(); + .build(); toPersist.add(new MapBasedInputRow(1, dims, event)); final File tmpDir = temporaryFolder.newFolder(); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java index 4baaddfde6fc..b1866ff0f29c 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java @@ -35,6 +35,7 @@ import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; @@ -133,7 +134,7 @@ public IndexMergerV9CompatibilityTest(SegmentWriteOutMediumFactory segmentWriteO @Before public void setUp() throws IOException { - toPersist = new IncrementalIndex.Builder() + toPersist = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(JodaUtils.MIN_INSTANT) @@ -141,7 +142,7 @@ public void setUp() throws IOException .build() ) .setMaxRowCount(1000000) - .buildOnheap(); + .build(); toPersist.getMetadata().put("key", "value"); for (InputRow event : events) { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java index ccc6ff7b359d..5358f28ed8a1 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -47,6 +47,7 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -101,7 +102,7 @@ public static Collection constructorFeeder() throws IOException private static IncrementalIndex makeIncrementalIndex() throws IOException { - IncrementalIndex theIndex = new IncrementalIndex.Builder() + IncrementalIndex theIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -126,7 +127,7 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); theIndex.add( new MapBasedInputRow( @@ -276,7 +277,7 @@ private static QueryableIndex makeMergedQueryableIndex( ) { try { - IncrementalIndex first = new IncrementalIndex.Builder() + IncrementalIndex first = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -301,9 +302,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); - IncrementalIndex second = new IncrementalIndex.Builder() + IncrementalIndex second = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -328,9 +329,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); - IncrementalIndex third = new IncrementalIndex.Builder() + IncrementalIndex third = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -355,7 +356,7 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); first.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java index 0cf4d0dd134b..7e0577ff6516 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.timeline.Overshadowable; @@ -149,7 +150,7 @@ public static QueryableIndex getIncrementalIndex(int index1, int index2) final long timestamp = new DateTime(event.get(TIMESTAMP), ISOChronology.getInstanceUTC()).getMillis(); if (theIndex == null) { - theIndex = new IncrementalIndex.Builder() + theIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(timestamp) @@ -158,7 +159,7 @@ public static QueryableIndex getIncrementalIndex(int index1, int index2) .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } final List dims = new ArrayList<>(); @@ -367,7 +368,7 @@ private void makeRowPersistedIndexes() } } - final IncrementalIndex rowIndex = new IncrementalIndex.Builder() + final IncrementalIndex rowIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(timestamp) @@ -376,7 +377,7 @@ private void makeRowPersistedIndexes() .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); rowIndex.add( new MapBasedInputRow(timestamp, dims, event) @@ -404,7 +405,7 @@ public static IncrementalIndex makeIncrementalIndex(final String resourceFilenam String filename = resource.getFile(); log.info("Realtime loading index file[%s]", filename); - final IncrementalIndex retVal = new IncrementalIndex.Builder() + final IncrementalIndex retVal = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) @@ -413,7 +414,7 @@ public static IncrementalIndex makeIncrementalIndex(final String resourceFilenam .build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); try { final List events = JSON_MAPPER.readValue(new File(filename), List.class); diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index f7baecc3acb2..10ed382531a9 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -49,6 +49,7 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -292,10 +293,10 @@ public static IncrementalIndex makeRealtimeIndex(final CharSource source, boolea .withMetrics(METRIC_AGGS) .withRollup(rollup) .build(); - final IncrementalIndex retVal = new IncrementalIndex.Builder() + final IncrementalIndex retVal = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .buildOnheap(); + .build(); try { return loadIncrementalIndex(retVal, source); 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 dfc3e533b289..534034b29bd8 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 @@ -63,9 +63,10 @@ import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.IncrementalIndex.Builder; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OffheapIncrementalIndex; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; import org.junit.AfterClass; @@ -130,10 +131,11 @@ public static Collection constructorFeeder() RESOURCE_CLOSER.register(pool1); params.add( new Object[] { - (IndexCreator) factories -> new Builder() + (IndexCreator) factories -> new OffheapIncrementalIndex.Builder() + .setBufferPool(pool1) .setSimpleTestingIndexSchema(factories) .setMaxRowCount(1000000) - .buildOffheap(pool1) + .build() } ); params.add(new Object[] {(IndexCreator) IncrementalIndexTest::createNoRollupIndex}); @@ -144,7 +146,8 @@ public static Collection constructorFeeder() RESOURCE_CLOSER.register(pool2); params.add( new Object[] { - (IndexCreator) factories -> new Builder() + (IndexCreator) factories -> new OffheapIncrementalIndex.Builder() + .setBufferPool(pool2) .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(factories) @@ -152,7 +155,7 @@ public static Collection constructorFeeder() .build() ) .setMaxRowCount(1000000) - .buildOffheap(pool2) + .build() } ); @@ -173,7 +176,7 @@ public static IncrementalIndex createIndex( aggregatorFactories = DEFAULT_AGGREGATOR_FACTORIES; } - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(dimensionsSpec) @@ -181,7 +184,7 @@ public static IncrementalIndex createIndex( .build() ) .setMaxRowCount(1000000) - .buildOnheap(); + .build(); } public static IncrementalIndex createIndex(AggregatorFactory[] aggregatorFactories) @@ -190,10 +193,10 @@ public static IncrementalIndex createIndex(AggregatorFactory[] aggregatorFactori aggregatorFactories = DEFAULT_AGGREGATOR_FACTORIES; } - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(aggregatorFactories) .setMaxRowCount(1000000) - .buildOnheap(); + .build(); } public static IncrementalIndex createNoRollupIndex(AggregatorFactory[] aggregatorFactories) @@ -202,10 +205,10 @@ public static IncrementalIndex createNoRollupIndex(AggregatorFactory[] aggregato aggregatorFactories = DEFAULT_AGGREGATOR_FACTORIES; } - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(false, aggregatorFactories) .setMaxRowCount(1000000) - .buildOnheap(); + .build(); } public static void populateIndex(long timestamp, IncrementalIndex index) throws IndexSizeExceededException @@ -722,7 +725,7 @@ public void run() @Test public void testgetDimensions() { - final IncrementalIndex incrementalIndex = new IncrementalIndex.Builder() + final IncrementalIndex incrementalIndex = (OnheapIncrementalIndex) new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(new CountAggregatorFactory("count")) @@ -736,7 +739,7 @@ public void testgetDimensions() .build() ) .setMaxRowCount(1000000) - .buildOnheap(); + .build(); closerRule.closeLater(incrementalIndex); Assert.assertEquals(Arrays.asList("dim0", "dim1"), incrementalIndex.getDimensionNames()); @@ -745,10 +748,10 @@ public void testgetDimensions() @Test public void testDynamicSchemaRollup() throws IndexSizeExceededException { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = (OnheapIncrementalIndex) new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(/* empty */) .setMaxRowCount(10) - .buildOnheap(); + .build(); closerRule.closeLater(index); index.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java index a6207aa09e52..5816ce922875 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java @@ -56,6 +56,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Interval; import org.junit.Test; @@ -114,7 +115,7 @@ public static Collection constructorFeeder() throws IOException private static IncrementalIndex makeIncrementalIndex() throws IOException { - IncrementalIndex theIndex = new IncrementalIndex.Builder() + IncrementalIndex theIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -134,7 +135,7 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); theIndex.add( new MapBasedInputRow( @@ -261,7 +262,7 @@ private static QueryableIndex makeMergedQueryableIndex( ) { try { - IncrementalIndex first = new IncrementalIndex.Builder() + IncrementalIndex first = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -282,9 +283,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); - IncrementalIndex second = new IncrementalIndex.Builder() + IncrementalIndex second = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -304,9 +305,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); - IncrementalIndex third = new IncrementalIndex.Builder() + IncrementalIndex third = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -327,7 +328,7 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); first.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java index 744b25d812f3..b9b1ba0ae519 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java @@ -55,6 +55,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -113,7 +114,7 @@ public static Collection constructorFeeder() throws IOException private static IncrementalIndex makeIncrementalIndex() throws IOException { - IncrementalIndex theIndex = new IncrementalIndex.Builder() + IncrementalIndex theIndex = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -138,7 +139,7 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); theIndex.add( new MapBasedInputRow( @@ -279,7 +280,7 @@ private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOE private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) { try { - IncrementalIndex first = new IncrementalIndex.Builder() + IncrementalIndex first = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -303,9 +304,9 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) ).build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); - IncrementalIndex second = new IncrementalIndex.Builder() + IncrementalIndex second = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -329,9 +330,9 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) ).build() ) .setMaxRowCount(1000) - .buildOnheap(); + .build(); - IncrementalIndex third = new IncrementalIndex.Builder() + IncrementalIndex third = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -355,7 +356,7 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) ).build() ) .setMaxRowCount(NUM_POINTS) - .buildOnheap(); + .build(); first.add( new MapBasedInputRow( 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 dfd386f3e92e..e678bf9fca18 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 @@ -78,10 +78,10 @@ public Object get(Object key) return null; } }; - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .buildOnheap(); + .build(); index.add( new MapBasedInputRow( 0, diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java index f6f95e02cef5..2d6320101b7e 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java @@ -38,10 +38,10 @@ public class IncrementalIndexRowCompTest extends InitializedNullHandlingTest @Test public void testBasic() { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(1000) - .buildOnheap(); + .build(); long time = System.currentTimeMillis(); IncrementalIndexRow ir1 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java index afcdfd492277..c8cc8a8d59da 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java @@ -37,11 +37,11 @@ public class IncrementalIndexRowSizeTest extends InitializedNullHandlingTest @Test public void testIncrementalIndexRowSizeBasic() { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(10000) .setMaxBytesInMemory(1000) - .buildOnheap(); + .build(); long time = System.currentTimeMillis(); IncrementalIndex.IncrementalIndexRowResult tndResult = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")); IncrementalIndexRow td1 = tndResult.getIncrementalIndexRow(); @@ -51,11 +51,11 @@ public void testIncrementalIndexRowSizeBasic() @Test public void testIncrementalIndexRowSizeArr() { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(10000) .setMaxBytesInMemory(1000) - .buildOnheap(); + .build(); long time = System.currentTimeMillis(); IncrementalIndex.IncrementalIndexRowResult tndResult = index.toIncrementalIndexRow(toMapRow( time + 1, @@ -71,11 +71,11 @@ public void testIncrementalIndexRowSizeArr() @Test public void testIncrementalIndexRowSizeComplex() { - IncrementalIndex index = new IncrementalIndex.Builder() + IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(10000) .setMaxBytesInMemory(1000) - .buildOnheap(); + .build(); long time = System.currentTimeMillis(); IncrementalIndex.IncrementalIndexRowResult tndResult = index.toIncrementalIndexRow(toMapRow( time + 1, diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index ec03a9e3d71d..f89a6f955cc3 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -111,10 +111,10 @@ public static Collection constructorFeeder() @Override public IncrementalIndex createIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } } } 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 91863ff54344..c7b885aee783 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 @@ -117,12 +117,12 @@ public static Collection constructorFeeder() @Override public IncrementalIndex createIndex() { - return new IncrementalIndex.Builder() + return new OnheapIncrementalIndex.Builder() .setIndexSchema(schema) .setDeserializeComplexMetrics(false) .setSortFacts(sortFacts) .setMaxRowCount(1000) - .buildOnheap(); + .build(); } }, Closer.create() @@ -141,11 +141,12 @@ public IncrementalIndex createIndex() @Override public IncrementalIndex createIndex() { - return new IncrementalIndex.Builder() + return new OffheapIncrementalIndex.Builder() + .setBufferPool(stupidPool) .setIndexSchema(schema) .setSortFacts(sortFacts) .setMaxRowCount(1000000) - .buildOffheap(stupidPool); + .build(); } }, poolCloser diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java index d475a5779f43..d1368e148680 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java @@ -47,7 +47,7 @@ public class OnheapIncrementalIndexTest extends InitializedNullHandlingTest @Test public void testMultithreadAddFacts() throws Exception { - final IncrementalIndex index = new IncrementalIndex.Builder() + final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -55,7 +55,7 @@ public void testMultithreadAddFacts() throws Exception .build() ) .setMaxRowCount(MAX_ROWS) - .buildOnheap(); + .build(); final int addThreadCount = 2; Thread[] addThreads = new Thread[addThreadCount]; @@ -111,7 +111,7 @@ public void run() @Test public void testMultithreadAddFactsUsingExpressionAndJavaScript() throws Exception { - final IncrementalIndex indexExpr = new IncrementalIndex.Builder() + final IncrementalIndex indexExpr = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -125,9 +125,9 @@ public void testMultithreadAddFactsUsingExpressionAndJavaScript() throws Excepti .build() ) .setMaxRowCount(MAX_ROWS) - .buildOnheap(); + .build(); - final IncrementalIndex indexJs = new IncrementalIndex.Builder() + final IncrementalIndex indexJs = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -143,7 +143,7 @@ public void testMultithreadAddFactsUsingExpressionAndJavaScript() throws Excepti .build() ) .setMaxRowCount(MAX_ROWS) - .buildOnheap(); + .build(); final int addThreadCount = 2; Thread[] addThreads = new Thread[addThreadCount]; @@ -205,7 +205,7 @@ public void testOnHeapIncrementalIndexClose() throws Exception mockedAggregator.close(); EasyMock.expectLastCall().times(1); - final OnheapIncrementalIndex index = (OnheapIncrementalIndex) new IncrementalIndex.Builder() + final OnheapIncrementalIndex index = (OnheapIncrementalIndex) new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -213,7 +213,7 @@ public void testOnHeapIncrementalIndexClose() throws Exception .build() ) .setMaxRowCount(MAX_ROWS) - .buildOnheap(); + .build(); index.add(new MapBasedInputRow( 0, 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 5bd1a5963711..64da13d8cc41 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 @@ -48,6 +48,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -360,7 +361,7 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept true ); - IncrementalIndex index = new IncrementalIndex.Builder().setMaxRowCount(100).setIndexSchema(schema).buildOnheap(); + IncrementalIndex index = new OnheapIncrementalIndex.Builder().setMaxRowCount(100).setIndexSchema(schema).build(); index.add( new MapBasedInputRow( DateTimes.nowUtc().getMillis(), diff --git a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java index 728e2ffac57b..948d60fc067d 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java @@ -25,6 +25,7 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig; import org.apache.druid.segment.realtime.plumber.IntervalStartVersioningPolicy; import org.apache.druid.segment.realtime.plumber.RejectionPolicyFactory; @@ -65,6 +66,7 @@ private static File createNewBasePersistDirectory() public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File basePersistDirectory) { return new RealtimeTuningConfig( + DEFAULT_APPENDABLE_INDEX, DEFAULT_MAX_ROWS_IN_MEMORY, 0L, DEFAULT_INTERMEDIATE_PERSIST_PERIOD, @@ -87,6 +89,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File ); } + private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; private final long maxBytesInMemory; private final Period intermediatePersistPeriod; @@ -110,6 +113,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File @JsonCreator public RealtimeTuningConfig( + @JsonProperty("appendableIndexSpec") @Nullable AppendableIndexSpec appendableIndexSpec, @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, @JsonProperty("maxBytesInMemory") Long maxBytesInMemory, @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, @@ -132,6 +136,7 @@ public RealtimeTuningConfig( @JsonProperty("dedupColumn") @Nullable String dedupColumn ) { + this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; // initializing this to 0, it will be lazily initialized to a value // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) @@ -166,6 +171,13 @@ public RealtimeTuningConfig( this.dedupColumn = dedupColumn == null ? DEFAULT_DEDUP_COLUMN : dedupColumn; } + @Override + @JsonProperty + public AppendableIndexSpec getAppendableIndexSpec() + { + return appendableIndexSpec; + } + @Override @JsonProperty public int getMaxRowsInMemory() @@ -304,6 +316,7 @@ public String getDedupColumn() public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, intermediatePersistPeriod, @@ -330,6 +343,7 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) public RealtimeTuningConfig withBasePersistDirectory(File dir) { return new RealtimeTuningConfig( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, intermediatePersistPeriod, diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java index 6d02ca42ac1a..24423c48d7a2 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java @@ -21,7 +21,10 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.utils.JvmUtils; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.AppendableIndexSpec; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; /** */ @@ -32,11 +35,43 @@ public interface TuningConfig { boolean DEFAULT_LOG_PARSE_EXCEPTIONS = false; + AppendableIndexSpec DEFAULT_APPENDABLE_INDEX = new OnheapIncrementalIndex.Spec(); int DEFAULT_MAX_PARSE_EXCEPTIONS = Integer.MAX_VALUE; int DEFAULT_MAX_SAVED_PARSE_EXCEPTIONS = 0; int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; - // We initially estimated this to be 1/3(max jvm memory), but bytesCurrentlyInMemory only - // tracks active index and not the index being flushed to disk, to account for that - // we halved default to 1/6(max jvm memory) - long DEFAULT_MAX_BYTES_IN_MEMORY = JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() / 6; + + /** + * The inceremental index implementation to use + */ + AppendableIndexSpec getAppendableIndexSpec(); + + /** + * Maximum number of bytes (estimated) to store in memory before persisting to local storage + */ + long getMaxBytesInMemory(); + + /** + * Maximum number of bytes (estimated) to store in memory before persisting to local storage. + * If getMaxBytesInMemory() returns 0, the appendable index default will be used. + */ + default long getMaxBytesInMemoryOrDefault() + { + // In the main tuningConfig class constructor, we set the maxBytes to 0 if null to avoid setting + // maxBytes to max jvm memory of the process that starts first. Instead we set the default based on + // the actual task node's jvm memory. + final long maxBytesInMemory = getMaxBytesInMemory(); + if (maxBytesInMemory > 0) { + return maxBytesInMemory; + } else if (maxBytesInMemory == 0) { + return getAppendableIndexSpec().getDefaultMaxBytesInMemory(); + } else { + return Long.MAX_VALUE; + } + } + + PartitionsSpec getPartitionsSpec(); + + IndexSpec getIndexSpec(); + + IndexSpec getIndexSpecForIntermediatePersists(); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java index f8d09e0ec415..ec760b22c22e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -19,15 +19,14 @@ package org.apache.druid.segment.realtime.appenderator; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; import javax.annotation.Nullable; import java.io.File; -public interface AppenderatorConfig +public interface AppenderatorConfig extends TuningConfig { boolean isReportParseExceptions(); @@ -36,11 +35,6 @@ public interface AppenderatorConfig */ int getMaxRowsInMemory(); - /** - * Maximum number of bytes (estimated) to store in memory before persisting to local storage - */ - long getMaxBytesInMemory(); - int getMaxPendingPersists(); /** @@ -61,17 +55,11 @@ default Long getMaxTotalRows() throw new UnsupportedOperationException("maxTotalRows is not implemented."); } - PartitionsSpec getPartitionsSpec(); - /** * Period that sets frequency to persist to local storage if no other thresholds are met */ Period getIntermediatePersistPeriod(); - IndexSpec getIndexSpec(); - - IndexSpec getIndexSpecForIntermediatePersists(); - File getBasePersistDirectory(); AppenderatorConfig withBasePersistDirectory(File basePersistDirectory); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index c65c2ad7cf0b..c2888376bd24 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -64,7 +64,6 @@ import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.TuningConfigs; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.realtime.FireDepartmentMetrics; import org.apache.druid.segment.realtime.FireHydrant; @@ -199,7 +198,7 @@ public class AppenderatorImpl implements Appenderator this.sinkTimeline = sinkQuerySegmentWalker.getSinkTimeline(); } - maxBytesTuningConfig = TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()); + maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); } @Override @@ -404,6 +403,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) schema, identifier.getShardSpec(), identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, null @@ -1121,6 +1121,7 @@ private Object bootstrapSinksFromDisk() schema, identifier.getShardSpec(), identifier.getVersion(), + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, null, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java index 7fae74c327ff..02f75c79900d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManager.java @@ -49,6 +49,7 @@ import org.apache.druid.segment.IndexableAdapter; import org.apache.druid.segment.ProgressIndicator; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.ParseExceptionHandler; import org.apache.druid.segment.incremental.RowIngestionMeters; @@ -382,6 +383,12 @@ public boolean isReportParseExceptions() return baseConfig.isReportParseExceptions(); } + @Override + public AppendableIndexSpec getAppendableIndexSpec() + { + return baseConfig.getAppendableIndexSpec(); + } + @Override public int getMaxRowsInMemory() { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index bae600c8b310..2aec7587a1bc 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -61,7 +61,6 @@ import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.TuningConfigs; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.realtime.FireDepartmentMetrics; @@ -260,8 +259,9 @@ private Sink getSink(long timestamp) schema, config.getShardSpec(), versioningPolicy.getVersion(sinkInterval), + config.getAppendableIndexSpec(), config.getMaxRowsInMemory(), - TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), + config.getMaxBytesInMemoryOrDefault(), config.getDedupColumn() ); addSink(retVal); @@ -723,8 +723,9 @@ public int compare(File o1, File o2) schema, config.getShardSpec(), versioningPolicy.getVersion(sinkInterval), + config.getAppendableIndexSpec(), config.getMaxRowsInMemory(), - TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), + config.getMaxBytesInMemoryOrDefault(), config.getDedupColumn(), hydrants ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index d6cb2230f180..ff20c1ee81bf 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IncrementalIndexSchema; @@ -63,6 +64,7 @@ public class Sink implements Iterable, Overshadowable private final DataSchema schema; private final ShardSpec shardSpec; private final String version; + private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; private final long maxBytesInMemory; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList<>(); @@ -79,6 +81,7 @@ public Sink( DataSchema schema, ShardSpec shardSpec, String version, + AppendableIndexSpec appendableIndexSpec, int maxRowsInMemory, long maxBytesInMemory, String dedupColumn @@ -89,6 +92,7 @@ public Sink( schema, shardSpec, version, + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, dedupColumn, @@ -101,6 +105,7 @@ public Sink( DataSchema schema, ShardSpec shardSpec, String version, + AppendableIndexSpec appendableIndexSpec, int maxRowsInMemory, long maxBytesInMemory, String dedupColumn, @@ -111,6 +116,7 @@ public Sink( this.shardSpec = shardSpec; this.interval = interval; this.version = version; + this.appendableIndexSpec = appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory; this.maxBytesInMemory = maxBytesInMemory; this.dedupColumn = dedupColumn; @@ -322,11 +328,11 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) .withMetrics(schema.getAggregators()) .withRollup(schema.getGranularitySpec().isRollup()) .build(); - final IncrementalIndex newIndex = new IncrementalIndex.Builder() + final IncrementalIndex newIndex = appendableIndexSpec.builder() .setIndexSchema(indexSchema) .setMaxRowCount(maxRowsInMemory) .setMaxBytesInMemory(maxBytesInMemory) - .buildOnheap(); + .build(); final FireHydrant old; synchronized (hydrantLock) { diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index ee79dc1f2286..87df0e8d532c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -62,6 +62,7 @@ public AppenderatorPlumberTest() throws Exception EasyMock.anyObject())).andReturn(true).anyTimes(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + null, 1, null, null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java index 9e57d2831316..c77d42991dbc 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -162,6 +162,7 @@ public AppenderatorTester( objectMapper ); tuningConfig = new RealtimeTuningConfig( + null, maxRowsInMemory, maxSizeInBytes == 0L ? getDefaultMaxBytesInMemory() : maxSizeInBytes, null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 9e9d350a30ab..6e38c1cbe614 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -134,6 +134,7 @@ public int columnCacheSizeBytes() ); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + null, 75000, null, null, diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index 1bb58ab40905..10a7ec91787c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -126,7 +127,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception try ( final QueryableIndex qi = indexIO.loadIndex(segmentDir); - final IncrementalIndex index = new IncrementalIndex.Builder() + final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(DIMENSIONS_SPEC_REINDEX) @@ -134,7 +135,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception .build() ) .setMaxRowCount(5000) - .buildOnheap() + .build() ) { final StorageAdapter sa = new QueryableIndexStorageAdapter(qi); final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); @@ -216,7 +217,7 @@ private void createTestIndex(File segmentDir) throws Exception ); try ( - final IncrementalIndex index = new IncrementalIndex.Builder() + final IncrementalIndex index = new OnheapIncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(parser.getParseSpec().getDimensionsSpec()) @@ -224,7 +225,7 @@ private void createTestIndex(File segmentDir) throws Exception .build() ) .setMaxRowCount(5000) - .buildOnheap() + .build() ) { for (String line : rows) { index.add(parser.parse(line)); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 0eb62e8ceca3..9b73cbe12cd3 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -48,7 +48,6 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.TuningConfigs; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.join.NoopJoinableFactory; import org.apache.druid.segment.loading.DataSegmentPusher; @@ -200,6 +199,7 @@ public void setUp() throws Exception EasyMock.replay(announcer, segmentPublisher, dataSegmentPusher, handoffNotifierFactory, handoffNotifier, emitter); tuningConfig = new RealtimeTuningConfig( + null, 1, null, null, @@ -278,8 +278,9 @@ private void testPersist(final Object commitMetadata) throws Exception schema, tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), - TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), + tuningConfig.getMaxBytesInMemoryOrDefault(), tuningConfig.getDedupColumn() ); plumber.getSinks().put(0L, sink); @@ -323,8 +324,9 @@ public void testPersistFails() throws Exception schema, tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), - TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), + tuningConfig.getMaxBytesInMemoryOrDefault(), tuningConfig.getDedupColumn() ); plumber.getSinks().put(0L, sink); @@ -373,8 +375,9 @@ private void testPersistHydrantGapsHelper(final Object commitMetadata) throws Ex schema2, tuningConfig.getShardSpec(), DateTimes.of("2014-12-01T12:34:56.789").toString(), + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), - TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), + tuningConfig.getMaxBytesInMemoryOrDefault(), tuningConfig.getDedupColumn() ); plumber2.getSinks().put(0L, sink); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java index bf94b2f70c89..477c3bd9b693 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java @@ -34,7 +34,6 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.RealtimeTuningConfig; -import org.apache.druid.segment.indexing.TuningConfigs; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -66,6 +65,7 @@ public void testSwap() throws Exception final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + null, 100, null, new Period("P1Y"), @@ -91,8 +91,9 @@ public void testSwap() throws Exception schema, tuningConfig.getShardSpec(), version, + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), - TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), + tuningConfig.getMaxBytesInMemoryOrDefault(), tuningConfig.getDedupColumn() ); @@ -220,6 +221,7 @@ public void testDedup() throws Exception final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final String version = DateTimes.nowUtc().toString(); RealtimeTuningConfig tuningConfig = new RealtimeTuningConfig( + null, 100, null, new Period("P1Y"), @@ -245,8 +247,9 @@ public void testDedup() throws Exception schema, tuningConfig.getShardSpec(), version, + tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), - TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), + tuningConfig.getMaxBytesInMemoryOrDefault(), tuningConfig.getDedupColumn() ); 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 bb759b6ddf7c..8dec45341c35 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 @@ -156,6 +156,7 @@ public void testTaskValidator() throws Exception ), new RealtimeTuningConfig( + null, 1, null, new Period("PT10M"), From a9fbe77ffdb4af4190d06e4e30bcd0258bc759e6 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Mon, 14 Sep 2020 10:00:55 +0300 Subject: [PATCH 02/20] Change to @UnstableApi --- .../apache/druid/segment/incremental/AppendableIndexSpec.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java index f9847ede47cd..6b5e135d0ae3 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java @@ -19,9 +19,9 @@ package org.apache.druid.segment.incremental; -import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.guice.annotations.UnstableApi; -@ExtensionPoint +@UnstableApi public interface AppendableIndexSpec { // Returns a builder of the appendable index. From e089d090097484989bcfb5c181b29ae722ec1c45 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Mon, 21 Sep 2020 13:13:23 +0300 Subject: [PATCH 03/20] Add AppendableIndexSpecTest --- .../incremental/AppendableIndexSpecTest.java | 71 +++++++++++++++++++ 1 file changed, 71 insertions(+) create mode 100644 processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java new file mode 100644 index 000000000000..84ce56bc4dd7 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java @@ -0,0 +1,71 @@ +/* + * 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.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +/** + */ +public class AppendableIndexSpecTest +{ + private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); + + @Test + public void testSerializationOnheapString() throws Exception + { + AppendableIndexSpec spec = JSON_MAPPER.readValue( + "{\"type\": \"onheap\"}", AppendableIndexSpec.class + ); + Assert.assertTrue(spec instanceof OnheapIncrementalIndex.Spec); + } + + @Test + public void testSerializationOffheapString() throws Exception + { + AppendableIndexSpec spec = JSON_MAPPER.readValue( + "{\"type\": \"offheap\"}", AppendableIndexSpec.class + ); + Assert.assertTrue(spec instanceof OffheapIncrementalIndex.Spec); + } + + @Test + public void testSerializationOffheapStringWithZeroSizes() throws Exception + { + AppendableIndexSpec spec = JSON_MAPPER.readValue( + "{\"type\": \"offheap\", \"bufferSize\": 0, \"cacheSize\": 0}", AppendableIndexSpec.class + ); + Assert.assertTrue(spec instanceof OffheapIncrementalIndex.Spec); + } + + @Test + public void testSerializationOffheapStringWithSizes() throws Exception + { + AppendableIndexSpec spec = JSON_MAPPER.readValue( + "{\"type\": \"offheap\", \"bufferSize\": 1048576, \"cacheSize\": 1073741824}", + AppendableIndexSpec.class + ); + Assert.assertTrue(spec instanceof OffheapIncrementalIndex.Spec); + Assert.assertEquals(1048576, ((OffheapIncrementalIndex.Spec) spec).bufferSize); + Assert.assertEquals(1073741824, ((OffheapIncrementalIndex.Spec) spec).cacheSize); + } +} From 1e5d8ba46ff8cbc575bdfbf27c1ca7ed56298c58 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Mon, 21 Sep 2020 13:28:59 +0300 Subject: [PATCH 04/20] Update doc --- docs/ingestion/index.md | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/docs/ingestion/index.md b/docs/ingestion/index.md index 071c8faecbab..fe8277afa9d6 100644 --- a/docs/ingestion/index.md +++ b/docs/ingestion/index.md @@ -712,7 +712,9 @@ is: "metricCompression": "lz4", "longEncoding": "longs" }, - "indexType": "onheap", + "appendableIndexSpec": { + "type": "onheap" + }, } ``` @@ -723,7 +725,7 @@ is: |maxRowsInMemory|The maximum number of records to store in memory before persisting to disk. Note that this is the number of rows post-rollup, and so it may not be equal to the number of input records. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).|`1000000`| |maxBytesInMemory|The maximum aggregate size of records, in bytes, to store in the JVM heap before persisting. This is based on a rough estimate of memory usage. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).

Setting maxBytesInMemory to -1 disables this check, meaning Druid will rely entirely on maxRowsInMemory to control memory usage. Setting it to zero means the default value will be used (one-sixth of JVM heap size).

Note that the estimate of memory usage is designed to be an overestimate, and can be especially high when using complex ingest-time aggregators, including sketches. If this causes your indexing workloads to persist to disk too often, you can set maxBytesInMemory to -1 and rely on maxRowsInMemory instead.|One-sixth of max JVM heap size| |indexSpec|Tune how data is indexed. See below for more information.|See table below| -|indexType|Specify which index type to use: `onheap`, or `offheap`.|`onheap`| +|appendableIndexSpec|Tune in-memory index. See below for more information.|See table below| |Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../development/extensions-core/kafka-ingestion.md#tuningconfig), [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md#tuningconfig), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).|| #### `indexSpec` @@ -739,3 +741,11 @@ The `indexSpec` object can include the following properties: Beyond these properties, each ingestion method has its own specific tuning properties. See the documentation for each [ingestion method](#ingestion-methods) for details. + +#### `appendableIndexSpec` + +|Field|Description|Default| +|-----|-----------|-------| +|type|Each in-memory index has its own tuning type code. You must specify the type code that matches your in-memory index. Common options are `onheap`, and `offheap`.|`onheap`| + +Beyond these properties, each in-memory index has its own specific tuning properties. From dea7e23f65f53df8c3a211d23ced441f50675fc9 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Sun, 4 Oct 2020 12:53:31 +0300 Subject: [PATCH 05/20] Add spelling exception --- website/.spelling | 1 + 1 file changed, 1 insertion(+) diff --git a/website/.spelling b/website/.spelling index a783d32ddc3b..96d5d5cc4390 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1772,6 +1772,7 @@ missingValue schemaless spatialDimensions useFieldDiscovery +appendableIndexSpec - ../docs/tutorials/index.md 4CPU cityName From efe35fd2a69b5c2ac7ae4381df4b4ad52380d6dd Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Sun, 4 Oct 2020 13:52:37 +0300 Subject: [PATCH 06/20] Add tests coverage --- .../MaterializedViewSupervisorTest.java | 29 ++++++++++++++++++- 1 file changed, 28 insertions(+), 1 deletion(-) diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 9694c2f4629e..afffd9a91c64 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -83,6 +83,7 @@ public class MaterializedViewSupervisorTest private TaskQueue taskQueue; private MaterializedViewSupervisor supervisor; private String derivativeDatasourceName; + private MaterializedViewSupervisorSpec spec; private final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); @Before @@ -103,7 +104,7 @@ public void setUp() taskQueue = EasyMock.createMock(TaskQueue.class); taskQueue.start(); objectMapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed")); - MaterializedViewSupervisorSpec spec = new MaterializedViewSupervisorSpec( + spec = new MaterializedViewSupervisorSpec( "base", new DimensionsSpec(Collections.singletonList(new StringDimensionSchema("dim")), null, null), new AggregatorFactory[]{new LongSumAggregatorFactory("m1", "m1")}, @@ -317,6 +318,32 @@ public void testCheckSegmentsAndSubmitTasks() throws IOException } + @Test + public void testCreateTask() + { + List baseSegments = Collections.singletonList( + new DataSegment( + "base", + Intervals.of("2015-01-02T00Z/2015-01-03T00Z"), + "2015-01-03", + ImmutableMap.of(), + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("m1"), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null, null), + 9, + 1024 + ) + ); + + HadoopIndexTask task = spec.createTask( + Intervals.of("2015-01-02T00Z/2015-01-03T00Z"), + "2015-01-03", + baseSegments + ); + + Assert.assertNotNull(task); + } + @Test public void testSuspendedDoesntRun() { From 452b196493dda7dd7c558bc67fa8db56e4b33395 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Tue, 6 Oct 2020 16:57:43 +0300 Subject: [PATCH 07/20] Revert some of the changes to reduce diff --- .../benchmark/FilterPartitionBenchmark.java | 5 +- .../FilteredAggregatorBenchmark.java | 5 +- .../GroupByTypeInterfaceBenchmark.java | 5 +- .../IncrementalIndexRowTypeBenchmark.java | 5 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 5 +- .../IncrementalIndexReadBenchmark.java | 5 +- .../indexing/IndexIngestionBenchmark.java | 5 +- .../indexing/IndexMergeBenchmark.java | 5 +- .../indexing/IndexPersistBenchmark.java | 5 +- .../benchmark/query/GroupByBenchmark.java | 5 +- .../druid/benchmark/query/ScanBenchmark.java | 5 +- .../benchmark/query/SearchBenchmark.java | 5 +- .../benchmark/query/TimeseriesBenchmark.java | 5 +- .../druid/benchmark/query/TopNBenchmark.java | 5 +- .../timecompare/TimeCompareBenchmark.java | 5 +- .../DistinctCountGroupByQueryTest.java | 5 +- .../DistinctCountTimeseriesQueryTest.java | 5 +- .../DistinctCountTopNQueryTest.java | 5 +- .../segment/MapVirtualColumnTestBase.java | 5 +- .../overlord/sampler/InputSourceSampler.java | 5 +- .../IngestSegmentFirehoseFactoryTest.java | 5 +- ...estSegmentFirehoseFactoryTimelineTest.java | 5 +- .../segment/incremental/IncrementalIndex.java | 59 ++++++++++++++ .../apache/druid/query/DoubleStorageTest.java | 5 +- .../druid/query/MultiValuedDimensionTest.java | 9 +-- .../aggregation/AggregationTestHelper.java | 13 ++-- .../first/StringFirstTimeseriesQueryTest.java | 5 +- .../last/StringLastTimeseriesQueryTest.java | 5 +- .../DataSourceMetadataQueryTest.java | 5 +- ...ByLimitPushDownInsufficientBufferTest.java | 5 +- ...roupByLimitPushDownMultiNodeMergeTest.java | 5 +- .../groupby/GroupByMultiSegmentTest.java | 5 +- .../GroupByQueryRunnerFactoryTest.java | 5 +- .../groupby/NestedQueryPushDownTest.java | 5 +- .../query/metadata/SegmentAnalyzerTest.java | 5 +- .../query/scan/MultiSegmentScanQueryTest.java | 5 +- .../query/search/SearchQueryRunnerTest.java | 5 +- .../TimeBoundaryQueryRunnerTest.java | 5 +- .../TimeseriesQueryRunnerBonusTest.java | 5 +- .../apache/druid/segment/EmptyIndexTest.java | 5 +- .../apache/druid/segment/IndexBuilder.java | 5 +- .../org/apache/druid/segment/IndexIOTest.java | 9 +-- .../druid/segment/IndexMergerTestBase.java | 77 +++++++++---------- .../IndexMergerV9CompatibilityTest.java | 5 +- .../IndexMergerV9WithSpatialIndexTest.java | 17 ++-- .../druid/segment/SchemalessIndexTest.java | 13 ++-- .../org/apache/druid/segment/TestIndex.java | 5 +- .../filter/SpatialFilterBonusTest.java | 17 ++-- .../segment/filter/SpatialFilterTest.java | 17 ++-- .../IncrementalIndexMultiValueSpecTest.java | 4 +- .../IncrementalIndexRowCompTest.java | 4 +- .../IncrementalIndexRowSizeTest.java | 12 +-- .../IncrementalIndexStorageAdapterTest.java | 4 +- .../OnheapIncrementalIndexTest.java | 16 ++-- .../virtual/ExpressionSelectorsTest.java | 3 +- .../firehose/IngestSegmentFirehoseTest.java | 9 +-- 56 files changed, 246 insertions(+), 237 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java index 6908b72909dc..f7a690fbb4af 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -70,7 +70,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.joda.time.Interval; @@ -228,10 +227,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java index c7eec63cb304..560148b41988 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java @@ -69,7 +69,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -226,10 +225,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex(AggregatorFactory[] metrics) { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(metrics) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query, String vectorize) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 19fe385a2926..1921b4359a32 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -70,7 +70,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -414,11 +413,11 @@ public String getFormatString() private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setConcurrentEventAdd(true) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } @TearDown(Level.Trial) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java index d228dfeb84a1..cc8d4a35d93e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IncrementalIndexRowTypeBenchmark.java @@ -29,7 +29,6 @@ import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Level; @@ -127,11 +126,11 @@ private MapBasedInputRow getStringRow(long timestamp, int dimensionCount) private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(aggs) .setDeserializeComplexMetrics(false) .setMaxRowCount(MAX_ROWS) - .build(); + .buildOnheap(); } @Setup diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java index b6078aecb5c4..fabb86ad4ff1 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -67,7 +67,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -309,10 +308,10 @@ public void setup() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index 9297792b4490..a49e122b2e7e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -46,7 +46,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -124,7 +123,7 @@ public void setup() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -132,7 +131,7 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java index f206e38d8654..84483582641c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java @@ -28,7 +28,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -105,7 +104,7 @@ public void setup2() private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -113,7 +112,7 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment * 2) - .build(); + .buildOnheap(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java index c683813c0a0c..b677a966a1c2 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -37,7 +37,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMediumFactory; @@ -212,7 +211,7 @@ private SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory(SegmentWrit private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -220,6 +219,6 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java index c23f810594e0..755947d7cbe5 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -34,7 +34,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.openjdk.jmh.annotations.Benchmark; @@ -152,7 +151,7 @@ public void teardown() private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMetrics(schemaInfo.getAggsArray()) @@ -160,7 +159,7 @@ private IncrementalIndex makeIncIndex() .build() ) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } @Benchmark diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 1f1ef83947ed..f212fe385293 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -85,7 +85,6 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -577,7 +576,7 @@ public String getFormatString() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(schemaInfo.getDimensionsSpec()) @@ -587,7 +586,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } @TearDown(Level.Trial) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 6248de2847a9..b5430769ce63 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -68,7 +68,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -320,10 +319,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java index 714f125b96a1..680a179fa05c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java @@ -75,7 +75,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -387,10 +386,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java index f7244cfdcc7d..875192ae4185 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java @@ -69,7 +69,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -311,10 +310,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java index f518289eb61a..b290f30de262 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -66,7 +66,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -293,10 +292,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java index 0208fe16f89e..406e62725b77 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java @@ -73,7 +73,6 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -405,10 +404,10 @@ public void tearDown() throws IOException private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setMaxRowCount(rowsPerSegment) - .build(); + .buildOnheap(); } @Benchmark diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java index 3da07ff62400..6c7db8eb3638 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java @@ -42,7 +42,6 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.After; import org.junit.Before; @@ -79,7 +78,7 @@ public void teardown() throws IOException @Test public void testGroupByWithDistinctCountAgg() throws Exception { - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -88,7 +87,7 @@ public void testGroupByWithDistinctCountAgg() throws Exception ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .build(); + .buildOnheap(); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java index f553bfcd1d63..2cc0526480bf 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java @@ -35,7 +35,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.junit.Test; @@ -51,7 +50,7 @@ public void testTopNWithDistinctCountAgg() throws Exception { TimeseriesQueryEngine engine = new TimeseriesQueryEngine(); - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -59,7 +58,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index ef1344c72bd0..7b14fba07f11 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -37,7 +37,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.junit.After; @@ -81,7 +80,7 @@ public void testTopNWithDistinctCountAgg() throws Exception { TopNQueryEngine engine = new TopNQueryEngine(pool); - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -89,7 +88,7 @@ public void testTopNWithDistinctCountAgg() throws Exception .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); String visitor_id = "visitor_id"; String client_type = "client_type"; diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java index 87286b4e0bf1..4d2164d7ec21 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnTestBase.java @@ -27,7 +27,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import java.io.IOException; @@ -63,10 +62,10 @@ static IncrementalIndex generateIndex() throws IOException .build(); return TestIndex.loadIncrementalIndex( - () -> new OnheapIncrementalIndex.Builder() + () -> new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .build(), + .buildOnheap(), input, parser ); 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 6cb1e59fd161..b0f470209023 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 @@ -43,7 +43,6 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.DataSchema; import javax.annotation.Nullable; @@ -211,8 +210,8 @@ private IncrementalIndex buildIncrementalIndex(SamplerConfig sampler .withRollup(dataSchema.getGranularitySpec().isRollup()) .build(); - return (OnheapIncrementalIndex) new OnheapIncrementalIndex.Builder().setIndexSchema(schema) + return new IncrementalIndex.Builder().setIndexSchema(schema) .setMaxRowCount(samplerConfig.getNumRows()) - .build(); + .buildOnheap(); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 01a1d2424e0d..6ab41617614c 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -73,7 +73,6 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.loading.LocalDataSegmentPuller; import org.apache.druid.segment.loading.LocalLoadSpec; import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory; @@ -197,10 +196,10 @@ public static Collection constructorFeeder() throws IOException new DoubleSumAggregatorFactory(METRIC_FLOAT_NAME, DIM_FLOAT_NAME) ) .build(); - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() + final IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(MAX_ROWS * MAX_SHARD_NUMBER) - .build(); + .buildOnheap(); for (Integer i = 0; i < MAX_ROWS; ++i) { index.add(ROW_PARSER.parseBatch(buildRow(i.longValue())).get(0)); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 400f7da3dbbe..06ebc56e9ec7 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -51,7 +51,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.timeline.DataSegment; @@ -251,10 +250,10 @@ private static Map persist(File tmpDir, InputRow... rows) .withDimensionsSpec(ROW_PARSER) .withMetrics(new LongSumAggregatorFactory(METRICS[0], METRICS[0])) .build(); - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() + final IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(rows.length) - .build(); + .buildOnheap(); for (InputRow row : rows) { try { 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 ccf0ea65dbdb..cc338b9854d0 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 @@ -316,6 +316,65 @@ protected IncrementalIndex( } } + /** + * This class exists only as backward competability to reduce the number of modified lines. + */ + public static class Builder extends OnheapIncrementalIndex.Builder + { + @Override + public Builder setIndexSchema(final IncrementalIndexSchema incrementalIndexSchema) + { + return (Builder) super.setIndexSchema(incrementalIndexSchema); + } + + @Override + public Builder setSimpleTestingIndexSchema(final AggregatorFactory... metrics) + { + return (Builder) super.setSimpleTestingIndexSchema(metrics); + } + + @Override + public Builder setSimpleTestingIndexSchema(@Nullable Boolean rollup, final AggregatorFactory... metrics) + { + return (Builder) super.setSimpleTestingIndexSchema(rollup, metrics); + } + + @Override + public Builder setDeserializeComplexMetrics(final boolean deserializeComplexMetrics) + { + return (Builder) super.setDeserializeComplexMetrics(deserializeComplexMetrics); + } + + @Override + public Builder setConcurrentEventAdd(final boolean concurrentEventAdd) + { + return (Builder) super.setConcurrentEventAdd(concurrentEventAdd); + } + + @Override + public Builder setSortFacts(final boolean sortFacts) + { + return (Builder) super.setSortFacts(sortFacts); + } + + @Override + public Builder setMaxRowCount(final int maxRowCount) + { + return (Builder) super.setMaxRowCount(maxRowCount); + } + + @Override + public Builder setMaxBytesInMemory(final long maxBytesInMemory) + { + return (Builder) super.setMaxBytesInMemory(maxBytesInMemory); + } + + public OnheapIncrementalIndex buildOnheap() + { + return (OnheapIncrementalIndex) build(); + } + } + public abstract FactsHolder getFacts(); diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index ee58422828e1..d6d27b88332a 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -55,7 +55,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -322,10 +321,10 @@ private static QueryableIndex buildIndex(String storeDoubleAsFloat) throws IOExc ) .build(); - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() + final IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(MAX_ROWS) - .build(); + .buildOnheap(); getStreamOfEvents().forEach(o -> { diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index be6b2072fb1a..33e16db52430 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -63,7 +63,6 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -141,10 +140,10 @@ public MultiValuedDimensionTest(final GroupByQueryConfig config, SegmentWriteOut @Before public void setup() throws Exception { - incrementalIndex = new OnheapIncrementalIndex.Builder() + incrementalIndex = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(5000) - .build(); + .buildOnheap(); StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( @@ -184,10 +183,10 @@ public void setup() throws Exception "UTF-8" ); - incrementalIndexNullSampler = new OnheapIncrementalIndex.Builder() + incrementalIndexNullSampler = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(5000) - .build(); + .buildOnheap(); String[] rowsNullSampler = new String[]{ "{\"time\":\"2011-01-13T00:00:00.000Z\",\"product\":\"product_1\",\"tags\":[],\"othertags\":[\"u1\", \"u2\"]}", diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java index d9b01c28a415..379b04a64a4b 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/AggregationTestHelper.java @@ -75,7 +75,6 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.junit.rules.TemporaryFolder; @@ -430,7 +429,7 @@ public void createIndex( List toMerge = new ArrayList<>(); try { - index = new OnheapIncrementalIndex.Builder() + index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -442,7 +441,7 @@ public void createIndex( ) .setDeserializeComplexMetrics(deserializeComplexMetrics) .setMaxRowCount(maxRowCount) - .build(); + .buildOnheap(); while (rows.hasNext()) { Object row = rows.next(); @@ -451,7 +450,7 @@ public void createIndex( toMerge.add(tmp); indexMerger.persist(index, tmp, new IndexSpec(), null); index.close(); - index = new OnheapIncrementalIndex.Builder() + index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -463,7 +462,7 @@ public void createIndex( ) .setDeserializeComplexMetrics(deserializeComplexMetrics) .setMaxRowCount(maxRowCount) - .build(); + .buildOnheap(); } if (row instanceof String && parser instanceof StringInputRowParser) { //Note: this is required because StringInputRowParser is InputRowParser as opposed to @@ -520,7 +519,7 @@ public static IncrementalIndex createIncrementalIndex( boolean rollup ) throws Exception { - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(minTimestamp) @@ -531,7 +530,7 @@ public static IncrementalIndex createIncrementalIndex( ) .setDeserializeComplexMetrics(deserializeComplexMetrics) .setMaxRowCount(maxRowCount) - .build(); + .buildOnheap(); while (rows.hasNext()) { Object row = rows.next(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index 3f833086d81d..445c6c9af419 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -42,7 +42,6 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; @@ -70,7 +69,7 @@ public void setUp() throws IndexSizeExceededException final SerializablePairLongStringSerde serde = new SerializablePairLongStringSerde(); ComplexMetrics.registerSerde(serde.getTypeName(), serde); - incrementalIndex = new OnheapIncrementalIndex.Builder() + incrementalIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -79,7 +78,7 @@ public void setUp() throws IndexSizeExceededException .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); incrementalIndex.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index ae79cb546b5a..7765ec4335a8 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -42,7 +42,6 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.junit.Before; @@ -69,7 +68,7 @@ public void setUp() throws IndexSizeExceededException final SerializablePairLongStringSerde serde = new SerializablePairLongStringSerde(); ComplexMetrics.registerSerde(serde.getTypeName(), serde); - incrementalIndex = new OnheapIncrementalIndex.Builder() + incrementalIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.SECOND) @@ -78,7 +77,7 @@ public void setUp() throws IndexSizeExceededException .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); incrementalIndex.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java index 5ac6b585220c..d22ba99efe94 100644 --- a/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/datasourcemetadata/DataSourceMetadataQueryTest.java @@ -41,7 +41,6 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.LogicalSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; @@ -115,10 +114,10 @@ public void testContextSerde() throws Exception @Test public void testMaxIngestedEventTime() throws Exception { - final IncrementalIndex rtIndex = new OnheapIncrementalIndex.Builder() + final IncrementalIndex rtIndex = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .build(); + .buildOnheap(); final QueryRunner runner = QueryRunnerTestHelper.makeQueryRunner( new DataSourceMetadataQueryRunnerFactory( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index f22701d8f69f..a2e994ea0abf 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -73,7 +73,6 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; @@ -134,7 +133,7 @@ public int columnCacheSizeBytes() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -150,7 +149,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index 3c62e11ea1a9..357a1e2bcd2a 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -79,7 +79,6 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -141,7 +140,7 @@ public int columnCacheSizeBytes() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -157,7 +156,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index 7b73cc149b59..4f9b1ffd4782 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -70,7 +70,6 @@ import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.junit.After; @@ -128,7 +127,7 @@ public int columnCacheSizeBytes() private IncrementalIndex makeIncIndex(boolean withRollup) { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -144,7 +143,7 @@ private IncrementalIndex makeIncIndex(boolean withRollup) ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java index 6f84d576599a..5c62fed9847c 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerFactoryTest.java @@ -42,7 +42,6 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.junit.After; import org.junit.Before; @@ -139,11 +138,11 @@ public Sequence run(QueryPlus queryPlus, ResponseContext responseContext) private Segment createSegment() throws Exception { - IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder() + IncrementalIndex incrementalIndex = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setConcurrentEventAdd(true) .setMaxRowCount(5000) - .build(); + .buildOnheap(); StringInputRowParser parser = new StringInputRowParser( new CSVParseSpec( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java index 8a97e891f3f5..9d793c73fd81 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/NestedQueryPushDownTest.java @@ -77,7 +77,6 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; import org.junit.After; @@ -126,7 +125,7 @@ public class NestedQueryPushDownTest private IncrementalIndex makeIncIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(new DimensionsSpec( @@ -143,7 +142,7 @@ private IncrementalIndex makeIncIndex() ) .setConcurrentEventAdd(true) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } @Before diff --git a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java index 11e00f76a505..6f56e0df94b1 100644 --- a/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java +++ b/processing/src/test/java/org/apache/druid/query/metadata/SegmentAnalyzerTest.java @@ -51,7 +51,6 @@ import org.apache.druid.segment.data.ObjectStrategy; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; @@ -276,10 +275,10 @@ public void testAnalyzingSegmentWithNonExistentAggregator() throws IOException .withRollup(true) .build(); - final IncrementalIndex retVal = new OnheapIncrementalIndex.Builder() + final IncrementalIndex retVal = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .build(); + .buildOnheap(); IncrementalIndex incrementalIndex = TestIndex.loadIncrementalIndex(retVal, source); QueryableIndex queryableIndex = TestIndex.persistRealtimeAndLoadMMapped(incrementalIndex); SegmentAnalyzer analyzer = new SegmentAnalyzer(EnumSet.of(SegmentMetadataQuery.AnalysisType.SIZE)); diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java index 93f46d313958..a579df09adf5 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java @@ -46,7 +46,6 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.NoneShardSpec; import org.joda.time.Interval; @@ -152,10 +151,10 @@ private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount) .withQueryGranularity(Granularities.HOUR) .withMetrics(TestIndex.METRIC_AGGS) .build(); - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(maxRowCount) - .build(); + .buildOnheap(); } @AfterClass diff --git a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java index 16a3590d9974..aebcf257b3e5 100644 --- a/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/search/SearchQueryRunnerTest.java @@ -59,7 +59,6 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.junit.Assert; @@ -720,14 +719,14 @@ public void testSearchOnFloatColumnWithExFn() @Test public void testSearchWithNullValueInDimension() throws Exception { - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) .build() ) .setMaxRowCount(10) - .build(); + .buildOnheap(); index.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java index 6834cec5e178..ecbe8af23768 100644 --- a/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeboundary/TimeBoundaryQueryRunnerTest.java @@ -41,7 +41,6 @@ import org.apache.druid.segment.TestIndex; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -118,10 +117,10 @@ private static IncrementalIndex newIndex(String minTimeStamp, int maxRowCount) .withQueryGranularity(Granularities.HOUR) .withMetrics(TestIndex.METRIC_AGGS) .build(); - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(maxRowCount) - .build(); + .buildOnheap(); } private static SegmentId makeIdentifier(IncrementalIndex index, String version) 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..a0d33e10ea42 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 @@ -38,7 +38,6 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.junit.Assert; import org.junit.Test; @@ -67,14 +66,14 @@ public TimeseriesQueryRunnerBonusTest(boolean descending) @Test public void testOneRowAtATime() throws Exception { - final IncrementalIndex oneRowIndex = new OnheapIncrementalIndex.Builder() + final IncrementalIndex oneRowIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2012-01-01T00:00:00Z").getMillis()) .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); List> results; diff --git a/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java b/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java index 306cb921b300..e9e8a17e6b21 100644 --- a/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/EmptyIndexTest.java @@ -28,7 +28,6 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAdapter; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; @@ -73,10 +72,10 @@ public void testEmptyIndex() throws Exception } try { - IncrementalIndex emptyIndex = new OnheapIncrementalIndex.Builder() + IncrementalIndex emptyIndex = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(/* empty */) .setMaxRowCount(1000) - .build(); + .buildOnheap(); IncrementalIndexAdapter emptyIndexAdapter = new IncrementalIndexAdapter( Intervals.of("2012-08-01/P3D"), diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index be2f36399660..5813f6d298ec 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -32,7 +32,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.timeline.SegmentId; @@ -226,10 +225,10 @@ private static IncrementalIndex buildIncrementalIndexWithRows( ) { Preconditions.checkNotNull(schema, "schema"); - final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex.Builder() + final IncrementalIndex incrementalIndex = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(maxRows) - .build(); + .buildOnheap(); for (InputRow row : rows) { try { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java index 5066bef4ac54..2fb6bc39eaa4 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIOTest.java @@ -40,7 +40,6 @@ import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; import org.junit.Assert; @@ -254,7 +253,7 @@ public IndexIOTest( this.exception = exception; } - final IncrementalIndex incrementalIndex1 = new OnheapIncrementalIndex.Builder() + final IncrementalIndex incrementalIndex1 = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DEFAULT_INTERVAL.getStart().getMillis()) @@ -269,9 +268,9 @@ public IndexIOTest( .build() ) .setMaxRowCount(1000000) - .build(); + .buildOnheap(); - final IncrementalIndex incrementalIndex2 = new OnheapIncrementalIndex.Builder() + final IncrementalIndex incrementalIndex2 = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DEFAULT_INTERVAL.getStart().getMillis()) @@ -286,7 +285,7 @@ public IndexIOTest( .build() ) .setMaxRowCount(1000000) - .build(); + .buildOnheap(); IndexableAdapter adapter1; IndexableAdapter adapter2; 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 40516f952396..86e8503f997f 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -57,7 +57,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -271,10 +270,10 @@ public void testPersistMerge() throws Exception IncrementalIndex toPersist1 = IncrementalIndexTest.createIndex(null); IncrementalIndexTest.populateIndex(timestamp, toPersist1); - IncrementalIndex toPersist2 = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersist2 = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersist2.add( new MapBasedInputRow( @@ -344,15 +343,15 @@ public void testPersistMerge() throws Exception @Test public void testPersistEmptyColumn() throws Exception { - final IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() + final IncrementalIndex toPersist1 = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(/* empty */) .setMaxRowCount(10) - .build(); + .buildOnheap(); - final IncrementalIndex toPersist2 = new OnheapIncrementalIndex.Builder() + final IncrementalIndex toPersist2 = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(/* empty */) .setMaxRowCount(10) - .build(); + .buildOnheap(); final File tmpDir1 = temporaryFolder.newFolder(); final File tmpDir2 = temporaryFolder.newFolder(); @@ -835,18 +834,18 @@ public void testMergeWithDimensionsList() throws Exception .build(); - IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersist1 = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .build(); - IncrementalIndex toPersist2 = new OnheapIncrementalIndex.Builder() + .buildOnheap(); + IncrementalIndex toPersist2 = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .build(); - IncrementalIndex toPersist3 = new OnheapIncrementalIndex.Builder() + .buildOnheap(); + IncrementalIndex toPersist3 = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); addDimValuesToIndex(toPersist1, "dimA", Arrays.asList("1", "2")); addDimValuesToIndex(toPersist2, "dimA", Arrays.asList("1", "2")); @@ -1020,10 +1019,10 @@ public void testJointDimMerge() throws Exception for (IncrementalIndexSchema indexSchema : Arrays.asList(rollupIndexSchema, noRollupIndexSchema)) { - IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistA = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersistA.add( new MapBasedInputRow( @@ -1040,10 +1039,10 @@ public void testJointDimMerge() throws Exception ) ); - IncrementalIndex toPersistB = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistB = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersistB.add( new MapBasedInputRow( @@ -1185,10 +1184,10 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception .withMetrics(new CountAggregatorFactory("count")) .withRollup(false) .build(); - IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistA = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersistA.add( new MapBasedInputRow( @@ -1209,10 +1208,10 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception ) ); - IncrementalIndex toPersistB = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistB = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersistB.add( new MapBasedInputRow( @@ -1321,10 +1320,10 @@ public void testMergeWithSupersetOrdering() throws Exception IncrementalIndex toPersistBA = getSingleDimIndex("dimB", Arrays.asList("1", "2", "3")); addDimValuesToIndex(toPersistBA, "dimA", Arrays.asList("1", "2")); - IncrementalIndex toPersistBA2 = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistBA2 = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersistBA2.add( new MapBasedInputRow( @@ -1868,10 +1867,10 @@ private IncrementalIndex getIndexWithDimsFromSchemata(List dims .withMetrics(new CountAggregatorFactory("count")) .build(); - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } @@ -1915,10 +1914,10 @@ public void testPersistNullColumnSkipping() throws Exception private IncrementalIndex getIndexD3() throws Exception { - IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersist1 = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersist1.add( new MapBasedInputRow( @@ -1949,10 +1948,10 @@ private IncrementalIndex getIndexD3() throws Exception private IncrementalIndex getSingleDimIndex(String dimName, List values) throws Exception { - IncrementalIndex toPersist1 = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersist1 = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) .setMaxRowCount(1000) - .build(); + .buildOnheap(); addDimValuesToIndex(toPersist1, dimName, values); return toPersist1; @@ -1972,10 +1971,10 @@ private IncrementalIndex getIndexWithDims(List dims) .withMetrics(new CountAggregatorFactory("count")) .build(); - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } private AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] aggregators) @@ -2208,10 +2207,10 @@ public void testMultivalDim_mergeAcrossSegments_rollupWorks() throws Exception .withRollup(true) .build(); - IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistA = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); Map event1 = new HashMap<>(); event1.put("dimA", "leek"); @@ -2226,10 +2225,10 @@ public void testMultivalDim_mergeAcrossSegments_rollupWorks() throws Exception toPersistA.add(new MapBasedInputRow(1, dims, event1)); toPersistA.add(new MapBasedInputRow(1, dims, event2)); - IncrementalIndex toPersistB = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistB = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); Map event3 = new HashMap<>(); event3.put("dimA", "leek"); @@ -2454,10 +2453,10 @@ public void testMultivalDim_persistAndMerge_dimensionValueOrderingRules() throws multivalEvent9 ); - IncrementalIndex toPersistA = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersistA = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); for (Map event : events) { toPersistA.add(new MapBasedInputRow(1, dims, event)); @@ -2470,10 +2469,10 @@ public void testMultivalDim_persistAndMerge_dimensionValueOrderingRules() throws List singleEventIndexes = new ArrayList<>(); for (Map event : events) { - IncrementalIndex toPersist = new OnheapIncrementalIndex.Builder() + IncrementalIndex toPersist = new IncrementalIndex.Builder() .setIndexSchema(indexSchema) .setMaxRowCount(1000) - .build(); + .buildOnheap(); toPersist.add(new MapBasedInputRow(1, dims, event)); final File tmpDir = temporaryFolder.newFolder(); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java index b1866ff0f29c..4baaddfde6fc 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9CompatibilityTest.java @@ -35,7 +35,6 @@ import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; @@ -134,7 +133,7 @@ public IndexMergerV9CompatibilityTest(SegmentWriteOutMediumFactory segmentWriteO @Before public void setUp() throws IOException { - toPersist = new OnheapIncrementalIndex.Builder() + toPersist = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(JodaUtils.MIN_INSTANT) @@ -142,7 +141,7 @@ public void setUp() throws IOException .build() ) .setMaxRowCount(1000000) - .build(); + .buildOnheap(); toPersist.getMetadata().put("key", "value"); for (InputRow event : events) { diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java index 5358f28ed8a1..ccc6ff7b359d 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -47,7 +47,6 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -102,7 +101,7 @@ public static Collection constructorFeeder() throws IOException private static IncrementalIndex makeIncrementalIndex() throws IOException { - IncrementalIndex theIndex = new OnheapIncrementalIndex.Builder() + IncrementalIndex theIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -127,7 +126,7 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); theIndex.add( new MapBasedInputRow( @@ -277,7 +276,7 @@ private static QueryableIndex makeMergedQueryableIndex( ) { try { - IncrementalIndex first = new OnheapIncrementalIndex.Builder() + IncrementalIndex first = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -302,9 +301,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); - IncrementalIndex second = new OnheapIncrementalIndex.Builder() + IncrementalIndex second = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -329,9 +328,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); - IncrementalIndex third = new OnheapIncrementalIndex.Builder() + IncrementalIndex third = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -356,7 +355,7 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); first.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java index 7e0577ff6516..0cf4d0dd134b 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessIndexTest.java @@ -39,7 +39,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.timeline.Overshadowable; @@ -150,7 +149,7 @@ public static QueryableIndex getIncrementalIndex(int index1, int index2) final long timestamp = new DateTime(event.get(TIMESTAMP), ISOChronology.getInstanceUTC()).getMillis(); if (theIndex == null) { - theIndex = new OnheapIncrementalIndex.Builder() + theIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(timestamp) @@ -159,7 +158,7 @@ public static QueryableIndex getIncrementalIndex(int index1, int index2) .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } final List dims = new ArrayList<>(); @@ -368,7 +367,7 @@ private void makeRowPersistedIndexes() } } - final IncrementalIndex rowIndex = new OnheapIncrementalIndex.Builder() + final IncrementalIndex rowIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(timestamp) @@ -377,7 +376,7 @@ private void makeRowPersistedIndexes() .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); rowIndex.add( new MapBasedInputRow(timestamp, dims, event) @@ -405,7 +404,7 @@ public static IncrementalIndex makeIncrementalIndex(final String resourceFilenam String filename = resource.getFile(); log.info("Realtime loading index file[%s]", filename); - final IncrementalIndex retVal = new OnheapIncrementalIndex.Builder() + final IncrementalIndex retVal = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DateTimes.of("2011-01-12T00:00:00.000Z").getMillis()) @@ -414,7 +413,7 @@ public static IncrementalIndex makeIncrementalIndex(final String resourceFilenam .build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); try { final List events = JSON_MAPPER.readValue(new File(filename), List.class); diff --git a/processing/src/test/java/org/apache/druid/segment/TestIndex.java b/processing/src/test/java/org/apache/druid/segment/TestIndex.java index 10ed382531a9..f7baecc3acb2 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestIndex.java +++ b/processing/src/test/java/org/apache/druid/segment/TestIndex.java @@ -49,7 +49,6 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -293,10 +292,10 @@ public static IncrementalIndex makeRealtimeIndex(final CharSource source, boolea .withMetrics(METRIC_AGGS) .withRollup(rollup) .build(); - final IncrementalIndex retVal = new OnheapIncrementalIndex.Builder() + final IncrementalIndex retVal = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .build(); + .buildOnheap(); try { return loadIncrementalIndex(retVal, source); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java index 5816ce922875..a6207aa09e52 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterBonusTest.java @@ -56,7 +56,6 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Interval; import org.junit.Test; @@ -115,7 +114,7 @@ public static Collection constructorFeeder() throws IOException private static IncrementalIndex makeIncrementalIndex() throws IOException { - IncrementalIndex theIndex = new OnheapIncrementalIndex.Builder() + IncrementalIndex theIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -135,7 +134,7 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); theIndex.add( new MapBasedInputRow( @@ -262,7 +261,7 @@ private static QueryableIndex makeMergedQueryableIndex( ) { try { - IncrementalIndex first = new OnheapIncrementalIndex.Builder() + IncrementalIndex first = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -283,9 +282,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); - IncrementalIndex second = new OnheapIncrementalIndex.Builder() + IncrementalIndex second = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -305,9 +304,9 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); - IncrementalIndex third = new OnheapIncrementalIndex.Builder() + IncrementalIndex third = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -328,7 +327,7 @@ private static QueryableIndex makeMergedQueryableIndex( ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); first.add( new MapBasedInputRow( diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java index b9b1ba0ae519..744b25d812f3 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SpatialFilterTest.java @@ -55,7 +55,6 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -114,7 +113,7 @@ public static Collection constructorFeeder() throws IOException private static IncrementalIndex makeIncrementalIndex() throws IOException { - IncrementalIndex theIndex = new OnheapIncrementalIndex.Builder() + IncrementalIndex theIndex = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -139,7 +138,7 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); theIndex.add( new MapBasedInputRow( @@ -280,7 +279,7 @@ private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOE private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) { try { - IncrementalIndex first = new OnheapIncrementalIndex.Builder() + IncrementalIndex first = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -304,9 +303,9 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) ).build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); - IncrementalIndex second = new OnheapIncrementalIndex.Builder() + IncrementalIndex second = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -330,9 +329,9 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) ).build() ) .setMaxRowCount(1000) - .build(); + .buildOnheap(); - IncrementalIndex third = new OnheapIncrementalIndex.Builder() + IncrementalIndex third = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withMinTimestamp(DATA_INTERVAL.getStartMillis()) @@ -356,7 +355,7 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) ).build() ) .setMaxRowCount(NUM_POINTS) - .build(); + .buildOnheap(); first.add( new MapBasedInputRow( 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 e678bf9fca18..dfd386f3e92e 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 @@ -78,10 +78,10 @@ public Object get(Object key) return null; } }; - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema(schema) .setMaxRowCount(10000) - .build(); + .buildOnheap(); index.add( new MapBasedInputRow( 0, diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java index 2d6320101b7e..f6f95e02cef5 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java @@ -38,10 +38,10 @@ public class IncrementalIndexRowCompTest extends InitializedNullHandlingTest @Test public void testBasic() { - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(1000) - .build(); + .buildOnheap(); long time = System.currentTimeMillis(); IncrementalIndexRow ir1 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java index c8cc8a8d59da..afcdfd492277 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowSizeTest.java @@ -37,11 +37,11 @@ public class IncrementalIndexRowSizeTest extends InitializedNullHandlingTest @Test public void testIncrementalIndexRowSizeBasic() { - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(10000) .setMaxBytesInMemory(1000) - .build(); + .buildOnheap(); long time = System.currentTimeMillis(); IncrementalIndex.IncrementalIndexRowResult tndResult = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")); IncrementalIndexRow td1 = tndResult.getIncrementalIndexRow(); @@ -51,11 +51,11 @@ public void testIncrementalIndexRowSizeBasic() @Test public void testIncrementalIndexRowSizeArr() { - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(10000) .setMaxBytesInMemory(1000) - .build(); + .buildOnheap(); long time = System.currentTimeMillis(); IncrementalIndex.IncrementalIndexRowResult tndResult = index.toIncrementalIndexRow(toMapRow( time + 1, @@ -71,11 +71,11 @@ public void testIncrementalIndexRowSizeArr() @Test public void testIncrementalIndexRowSizeComplex() { - IncrementalIndex index = new OnheapIncrementalIndex.Builder() + IncrementalIndex index = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(10000) .setMaxBytesInMemory(1000) - .build(); + .buildOnheap(); long time = System.currentTimeMillis(); IncrementalIndex.IncrementalIndexRowResult tndResult = index.toIncrementalIndexRow(toMapRow( time + 1, diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index f89a6f955cc3..ec03a9e3d71d 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -111,10 +111,10 @@ public static Collection constructorFeeder() @Override public IncrementalIndex createIndex() { - return new OnheapIncrementalIndex.Builder() + return new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(1000) - .build(); + .buildOnheap(); } } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java index d1368e148680..d475a5779f43 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java @@ -47,7 +47,7 @@ public class OnheapIncrementalIndexTest extends InitializedNullHandlingTest @Test public void testMultithreadAddFacts() throws Exception { - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() + final IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -55,7 +55,7 @@ public void testMultithreadAddFacts() throws Exception .build() ) .setMaxRowCount(MAX_ROWS) - .build(); + .buildOnheap(); final int addThreadCount = 2; Thread[] addThreads = new Thread[addThreadCount]; @@ -111,7 +111,7 @@ public void run() @Test public void testMultithreadAddFactsUsingExpressionAndJavaScript() throws Exception { - final IncrementalIndex indexExpr = new OnheapIncrementalIndex.Builder() + final IncrementalIndex indexExpr = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -125,9 +125,9 @@ public void testMultithreadAddFactsUsingExpressionAndJavaScript() throws Excepti .build() ) .setMaxRowCount(MAX_ROWS) - .build(); + .buildOnheap(); - final IncrementalIndex indexJs = new OnheapIncrementalIndex.Builder() + final IncrementalIndex indexJs = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -143,7 +143,7 @@ public void testMultithreadAddFactsUsingExpressionAndJavaScript() throws Excepti .build() ) .setMaxRowCount(MAX_ROWS) - .build(); + .buildOnheap(); final int addThreadCount = 2; Thread[] addThreads = new Thread[addThreadCount]; @@ -205,7 +205,7 @@ public void testOnHeapIncrementalIndexClose() throws Exception mockedAggregator.close(); EasyMock.expectLastCall().times(1); - final OnheapIncrementalIndex index = (OnheapIncrementalIndex) new OnheapIncrementalIndex.Builder() + final OnheapIncrementalIndex index = (OnheapIncrementalIndex) new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withQueryGranularity(Granularities.MINUTE) @@ -213,7 +213,7 @@ public void testOnHeapIncrementalIndexClose() throws Exception .build() ) .setMaxRowCount(MAX_ROWS) - .build(); + .buildOnheap(); index.add(new MapBasedInputRow( 0, 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 64da13d8cc41..5bd1a5963711 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 @@ -48,7 +48,6 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; import org.apache.druid.segment.incremental.IndexSizeExceededException; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; @@ -361,7 +360,7 @@ public void test_incrementalIndexStringSelector() throws IndexSizeExceededExcept true ); - IncrementalIndex index = new OnheapIncrementalIndex.Builder().setMaxRowCount(100).setIndexSchema(schema).build(); + IncrementalIndex index = new IncrementalIndex.Builder().setMaxRowCount(100).setIndexSchema(schema).buildOnheap(); index.add( new MapBasedInputRow( DateTimes.nowUtc().getMillis(), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index 10a7ec91787c..1bb58ab40905 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -46,7 +46,6 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; -import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -127,7 +126,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception try ( final QueryableIndex qi = indexIO.loadIndex(segmentDir); - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() + final IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(DIMENSIONS_SPEC_REINDEX) @@ -135,7 +134,7 @@ public void testReadFromIndexAndWriteAnotherIndex() throws Exception .build() ) .setMaxRowCount(5000) - .build() + .buildOnheap() ) { final StorageAdapter sa = new QueryableIndexStorageAdapter(qi); final WindowedStorageAdapter wsa = new WindowedStorageAdapter(sa, sa.getInterval()); @@ -217,7 +216,7 @@ private void createTestIndex(File segmentDir) throws Exception ); try ( - final IncrementalIndex index = new OnheapIncrementalIndex.Builder() + final IncrementalIndex index = new IncrementalIndex.Builder() .setIndexSchema( new IncrementalIndexSchema.Builder() .withDimensionsSpec(parser.getParseSpec().getDimensionsSpec()) @@ -225,7 +224,7 @@ private void createTestIndex(File segmentDir) throws Exception .build() ) .setMaxRowCount(5000) - .build() + .buildOnheap() ) { for (String line : rows) { index.add(parser.parse(line)); From 56ed0b595ac7318204c3355b216474f553fbf1d3 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Wed, 7 Oct 2020 21:31:31 +0300 Subject: [PATCH 08/20] Minor fixes --- .../java/org/apache/druid/query/groupby/GroupByQueryHelper.java | 2 +- .../druid/segment/incremental/AppendableIndexBuilder.java | 2 +- .../java/org/apache/druid/segment/indexing/TuningConfig.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java index 6e72cb495c83..f0b2a9e1edde 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryHelper.java @@ -121,7 +121,7 @@ public String apply(DimensionSpec input) .build(); - AppendableIndexBuilder indexBuilder; + final AppendableIndexBuilder indexBuilder; if (query.getContextValue("useOffheap", false)) { indexBuilder = new OffheapIncrementalIndex.Builder() diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java index 2cf6ccd2cc80..220f0e3a6506 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexBuilder.java @@ -119,7 +119,7 @@ public void validate() public final IncrementalIndex build() { - log.info("Building appendable index."); + log.debug("Building appendable index."); validate(); return buildInner(); } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java index 24423c48d7a2..e659c3973e43 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java @@ -41,7 +41,7 @@ public interface TuningConfig int DEFAULT_MAX_ROWS_IN_MEMORY = 1_000_000; /** - * The inceremental index implementation to use + * The incremental index implementation to use */ AppendableIndexSpec getAppendableIndexSpec(); From fb671417c04b1c540d8814be72e50eb40d3438fc Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Wed, 7 Oct 2020 21:34:00 +0300 Subject: [PATCH 09/20] Update getMaxBytesInMemoryOrDefault() comment --- .../java/org/apache/druid/indexer/HadoopTuningConfig.java | 4 ++-- .../common/index/RealtimeAppenderatorTuningConfig.java | 4 ++-- .../java/org/apache/druid/indexing/common/task/IndexTask.java | 4 ++-- .../seekablestream/SeekableStreamIndexTaskTuningConfig.java | 4 ++-- .../apache/druid/segment/indexing/RealtimeTuningConfig.java | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index 29a8b6edf97b..7e0c6deabb30 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -145,8 +145,8 @@ public HadoopTuningConfig( ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; - // initializing this to 0, it will be lazily initialized to a value - // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + /** initializing this to 0, it will be lazily intialized to a value + * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index 56c71fdb07fc..b4cdd22478fb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -98,8 +98,8 @@ public RealtimeAppenderatorTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - // initializing this to 0, it will be lazily intialized to a value - // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + /** initializing this to 0, it will be lazily intialized to a value + * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); this.intermediatePersistPeriod = intermediatePersistPeriod == null diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 39bb537d4bd6..bfb0da3a048f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1269,8 +1269,8 @@ private IndexTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - // initializing this to 0, it will be lazily initialized to a value - // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + /** initializing this to 0, it will be lazily intialized to a value + * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.partitionsSpec = partitionsSpec; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index d536db84c33b..9da264225219 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -90,8 +90,8 @@ public SeekableStreamIndexTaskTuningConfig( this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); - // initializing this to 0, it will be lazily initialized to a value - // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + /** initializing this to 0, it will be lazily intialized to a value + * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaults.getIntermediatePersistPeriod() diff --git a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java index 948d60fc067d..b3d4cf41835d 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java @@ -138,8 +138,8 @@ public RealtimeTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - // initializing this to 0, it will be lazily initialized to a value - // @see server.src.main.java.org.apache.druid.segment.indexing.TuningConfigs#getMaxBytesInMemoryOrDefault(long) + /** initializing this to 0, it will be lazily intialized to a value + * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? DEFAULT_INTERMEDIATE_PERSIST_PERIOD From 704a3317bf19e371f62c071f387fc4d325710d2e Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 8 Oct 2020 10:39:02 +0300 Subject: [PATCH 10/20] Fix typo, remove redundant interface --- .../java/org/apache/druid/indexer/HadoopTuningConfig.java | 2 +- .../common/index/RealtimeAppenderatorTuningConfig.java | 5 ++--- .../org/apache/druid/indexing/common/task/IndexTask.java | 4 ++-- .../seekablestream/SeekableStreamIndexTaskTuningConfig.java | 4 ++-- .../apache/druid/segment/indexing/RealtimeTuningConfig.java | 5 ++--- 5 files changed, 9 insertions(+), 11 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index 7e0c6deabb30..bdfe98334e43 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -145,7 +145,7 @@ public HadoopTuningConfig( ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; - /** initializing this to 0, it will be lazily intialized to a value + /** initializing this to 0, it will be lazily initialized to a value * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.leaveIntermediate = leaveIntermediate; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index b4cdd22478fb..5daf6cf03407 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -38,9 +38,8 @@ import java.io.File; @JsonTypeName("realtime_appenderator") -public class RealtimeAppenderatorTuningConfig implements TuningConfig, AppenderatorConfig +public class RealtimeAppenderatorTuningConfig implements AppenderatorConfig { - private static final int DEFAULT_MAX_ROWS_IN_MEMORY = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M"); private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; private static final ShardSpec DEFAULT_SHARD_SPEC = new NumberedShardSpec(0, 1); @@ -98,7 +97,7 @@ public RealtimeAppenderatorTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - /** initializing this to 0, it will be lazily intialized to a value + /** initializing this to 0, it will be lazily initialized to a value * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index bfb0da3a048f..ebfb304cf0d2 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1111,7 +1111,7 @@ public boolean isAppendToExisting() } } - public static class IndexTuningConfig implements TuningConfig, AppenderatorConfig + public static class IndexTuningConfig implements AppenderatorConfig { private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); private static final int DEFAULT_MAX_PENDING_PERSISTS = 0; @@ -1269,7 +1269,7 @@ private IndexTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - /** initializing this to 0, it will be lazily intialized to a value + /** initializing this to 0, it will be lazily initialized to a value * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.partitionsSpec = partitionsSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index 9da264225219..ff2a52f57eea 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -33,7 +33,7 @@ import java.io.File; import java.util.Objects; -public abstract class SeekableStreamIndexTaskTuningConfig implements TuningConfig, AppenderatorConfig +public abstract class SeekableStreamIndexTaskTuningConfig implements AppenderatorConfig { private static final boolean DEFAULT_RESET_OFFSET_AUTOMATICALLY = false; private static final boolean DEFAULT_SKIP_SEQUENCE_NUMBER_AVAILABILITY_CHECK = false; @@ -90,7 +90,7 @@ public SeekableStreamIndexTaskTuningConfig( this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); - /** initializing this to 0, it will be lazily intialized to a value + /** initializing this to 0, it will be lazily initialized to a value * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null diff --git a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java index b3d4cf41835d..44dbe82b749a 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java @@ -42,9 +42,8 @@ /** * */ -public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig +public class RealtimeTuningConfig implements AppenderatorConfig { - private static final int DEFAULT_MAX_ROWS_IN_MEMORY = TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY; private static final Period DEFAULT_INTERMEDIATE_PERSIST_PERIOD = new Period("PT10M"); private static final Period DEFAULT_WINDOW_PERIOD = new Period("PT10M"); private static final VersioningPolicy DEFAULT_VERSIONING_POLICY = new IntervalStartVersioningPolicy(); @@ -138,7 +137,7 @@ public RealtimeTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - /** initializing this to 0, it will be lazily intialized to a value + /** initializing this to 0, it will be lazily initialized to a value * @see #getMaxBytesInMemoryOrDefault() */ this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null From 948af29a050b2ef21410ba13bfc39d175d46242a Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 8 Oct 2020 10:51:35 +0300 Subject: [PATCH 11/20] Remove off-heap spec (postponed to a later PR) --- .../druid/jackson/AppendableIndexModule.java | 2 - .../incremental/OffheapIncrementalIndex.java | 65 ------------------- .../incremental/AppendableIndexSpecTest.java | 27 +------- 3 files changed, 3 insertions(+), 91 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java b/processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java index 3cc95879e95b..177c7a5f7847 100644 --- a/processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java +++ b/processing/src/main/java/org/apache/druid/jackson/AppendableIndexModule.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.databind.module.SimpleModule; import org.apache.druid.segment.incremental.AppendableIndexSpec; -import org.apache.druid.segment.incremental.OffheapIncrementalIndex; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; public class AppendableIndexModule extends SimpleModule @@ -38,7 +37,6 @@ public AppendableIndexModule() @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = OnheapIncrementalIndex.Spec.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = OnheapIncrementalIndex.Spec.TYPE, value = OnheapIncrementalIndex.Spec.class), - @JsonSubTypes.Type(name = OffheapIncrementalIndex.Spec.TYPE, value = OffheapIncrementalIndex.Spec.class) }) public interface AppendableIndexSpecMixin { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java index 406005f84676..b3cdabcd5e36 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java @@ -19,12 +19,9 @@ package org.apache.druid.segment.incremental; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; -import org.apache.druid.collections.StupidPool; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -35,7 +32,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.io.IOException; @@ -383,65 +379,4 @@ protected OffheapIncrementalIndex buildInner() ); } } - - public static class Spec implements AppendableIndexSpec, Supplier - { - public static final String TYPE = "offheap"; - static final int DEFAULT_BUFFER_SIZE = 1 << 23; - static final int DEFAULT_CACHE_SIZE = 1 << 30; - - final int bufferSize; - final int cacheSize; - final NonBlockingPool bufferPool; - - @JsonCreator - public Spec( - final @JsonProperty("bufferSize") @Nullable Integer bufferSize, - final @JsonProperty("cacheSize") @Nullable Integer cacheSize - ) - { - this.bufferSize = bufferSize != null && bufferSize > 0 ? bufferSize : DEFAULT_BUFFER_SIZE; - this.cacheSize = cacheSize != null && cacheSize > this.bufferSize ? cacheSize : DEFAULT_CACHE_SIZE; - this.bufferPool = new StupidPool<>( - "Offheap incremental-index buffer pool", - this, - 0, - this.cacheSize / this.bufferSize - ); - } - - @JsonProperty - public int getBufferSize() - { - return bufferSize; - } - - @JsonProperty - public int getCacheSize() - { - return cacheSize; - } - - @Override - public ByteBuffer get() - { - return ByteBuffer.allocateDirect(bufferSize); - } - - @Override - public AppendableIndexBuilder builder() - { - return new Builder().setBufferPool(bufferPool); - } - - @Override - public long getDefaultMaxBytesInMemory() - { - // In the realtime node, the entire JVM's direct memory is utilized for ingestion and persist operations. - // But maxBytesInMemory only refers to the active index size and not to the index being flushed to disk and the - // persist buffer. - // To account for that, we set default to 1/2 of the max jvm's direct memory. - return JvmUtils.getRuntimeInfo().getDirectMemorySizeBytes() / 2; - } - } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java index 84ce56bc4dd7..60cac8631d6c 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java @@ -40,32 +40,11 @@ public void testSerializationOnheapString() throws Exception } @Test - public void testSerializationOffheapString() throws Exception + public void testSerializationDefaultString() throws Exception { AppendableIndexSpec spec = JSON_MAPPER.readValue( - "{\"type\": \"offheap\"}", AppendableIndexSpec.class + "{}", AppendableIndexSpec.class ); - Assert.assertTrue(spec instanceof OffheapIncrementalIndex.Spec); - } - - @Test - public void testSerializationOffheapStringWithZeroSizes() throws Exception - { - AppendableIndexSpec spec = JSON_MAPPER.readValue( - "{\"type\": \"offheap\", \"bufferSize\": 0, \"cacheSize\": 0}", AppendableIndexSpec.class - ); - Assert.assertTrue(spec instanceof OffheapIncrementalIndex.Spec); - } - - @Test - public void testSerializationOffheapStringWithSizes() throws Exception - { - AppendableIndexSpec spec = JSON_MAPPER.readValue( - "{\"type\": \"offheap\", \"bufferSize\": 1048576, \"cacheSize\": 1073741824}", - AppendableIndexSpec.class - ); - Assert.assertTrue(spec instanceof OffheapIncrementalIndex.Spec); - Assert.assertEquals(1048576, ((OffheapIncrementalIndex.Spec) spec).bufferSize); - Assert.assertEquals(1073741824, ((OffheapIncrementalIndex.Spec) spec).cacheSize); + Assert.assertTrue(spec instanceof OnheapIncrementalIndex.Spec); } } From 6e9542ec97abfe528269c315f3ad27725ef79bec Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 8 Oct 2020 11:02:18 +0300 Subject: [PATCH 12/20] Add javadocs to AppendableIndexSpec --- .../apache/druid/segment/incremental/AppendableIndexSpec.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java index 6b5e135d0ae3..67cdabdf5673 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AppendableIndexSpec.java @@ -21,6 +21,9 @@ import org.apache.druid.guice.annotations.UnstableApi; +/** + * AppendableIndexSpec describes the in-memory indexing method for data ingestion. + */ @UnstableApi public interface AppendableIndexSpec { From 2a869d1711e51bee2e4b83296edf427aff7de01f Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 8 Oct 2020 11:13:01 +0300 Subject: [PATCH 13/20] Describe testCreateTask() --- .../materializedview/MaterializedViewSupervisorTest.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index afffd9a91c64..8dc6e53a4e2b 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -318,6 +318,9 @@ public void testCheckSegmentsAndSubmitTasks() throws IOException } + /** + * Verifies that creating HadoopIndexTask compleates without raising exception. + */ @Test public void testCreateTask() { From 98788ec97fc49bf2ee8ab97a9348b6a2f9b2684c Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 8 Oct 2020 12:59:25 +0300 Subject: [PATCH 14/20] Add tests for AppendableIndexSpec within TuningConfig --- .../kafka/KafkaIndexTaskTuningConfigTest.java | 9 +++- .../KafkaSupervisorTuningConfigTest.java | 6 ++- .../KinesisIndexTaskTuningConfigTest.java | 9 +++- .../KinesisSupervisorTuningConfigTest.java | 6 ++- .../druid/indexer/HadoopTuningConfigTest.java | 2 + .../druid/indexing/common/task/IndexTask.java | 3 +- .../SeekableStreamIndexTaskTuningConfig.java | 3 +- .../incremental/OnheapIncrementalIndex.java | 6 +++ .../incremental/AppendableIndexSpecTest.java | 50 ------------------- .../indexing/RealtimeTuningConfigTest.java | 6 ++- 10 files changed, 43 insertions(+), 57 deletions(-) delete mode 100644 processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 24845dcec155..9d38430acc49 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -26,6 +26,7 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.TuningConfig; import org.joda.time.Period; import org.junit.Assert; @@ -60,6 +61,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertNull(config.getMaxTotalRows()); @@ -85,7 +87,8 @@ public void testSerdeWithNonDefaults() throws Exception + " \"reportParseExceptions\": true,\n" + " \"handoffConditionTimeout\": 100,\n" + " \"indexSpec\": { \"metricCompression\" : \"NONE\" },\n" - + " \"indexSpecForIntermediatePersists\": { \"dimensionCompression\" : \"uncompressed\" }\n" + + " \"indexSpecForIntermediatePersists\": { \"dimensionCompression\" : \"uncompressed\" },\n" + + " \"appendableIndexSpec\": { \"type\" : \"onheap\" }\n" + "}"; KafkaIndexTaskTuningConfig config = (KafkaIndexTaskTuningConfig) mapper.readValue( @@ -99,6 +102,7 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment().intValue()); Assert.assertNotEquals(null, config.getMaxTotalRows()); @@ -143,6 +147,7 @@ public void testConvert() ); KafkaIndexTaskTuningConfig copy = (KafkaIndexTaskTuningConfig) original.convertToTaskTuningConfig(); + Assert.assertEquals(original.getAppendableIndexSpec(), copy.getAppendableIndexSpec()); Assert.assertEquals(1, copy.getMaxRowsInMemory()); Assert.assertEquals(2, copy.getMaxRowsPerSegment().intValue()); Assert.assertNotEquals(null, copy.getMaxTotalRows()); @@ -185,6 +190,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException mapper.readValue(serialized, TestModifiedKafkaIndexTaskTuningConfig.class); Assert.assertEquals(null, deserialized.getExtra()); + Assert.assertEquals(base.getAppendableIndexSpec(), deserialized.getAppendableIndexSpec()); Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); @@ -234,6 +240,7 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException KafkaIndexTaskTuningConfig deserialized = mapper.readValue(serialized, KafkaIndexTaskTuningConfig.class); + Assert.assertEquals(base.getAppendableIndexSpec(), deserialized.getAppendableIndexSpec()); Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java index 5859f9035b06..7151eb0681fb 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfigTest.java @@ -25,6 +25,7 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.TuningConfig; import org.joda.time.Duration; import org.joda.time.Period; @@ -59,6 +60,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); @@ -94,7 +96,8 @@ public void testSerdeWithNonDefaults() throws Exception + " \"shutdownTimeout\": \"PT95S\",\n" + " \"offsetFetchPeriod\": \"PT20S\",\n" + " \"indexSpec\": { \"metricCompression\" : \"NONE\" },\n" - + " \"indexSpecForIntermediatePersists\": { \"dimensionCompression\" : \"uncompressed\" }\n" + + " \"indexSpecForIntermediatePersists\": { \"dimensionCompression\" : \"uncompressed\" },\n" + + " \"appendableIndexSpec\": { \"type\" : \"onheap\" }\n" + "}"; KafkaSupervisorTuningConfig config = (KafkaSupervisorTuningConfig) mapper.readValue( @@ -108,6 +111,7 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 149200b59e8b..7a255966e41b 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -26,6 +26,7 @@ import org.apache.druid.indexing.kinesis.test.TestModifiedKinesisIndexTaskTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.TuningConfig; import org.hamcrest.CoreMatchers; import org.joda.time.Period; @@ -66,6 +67,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); @@ -102,7 +104,8 @@ public void testSerdeWithNonDefaults() throws Exception + " \"fetchSequenceNumberTimeout\": 6000,\n" + " \"resetOffsetAutomatically\": false,\n" + " \"skipSequenceNumberAvailabilityCheck\": true,\n" - + " \"fetchThreads\": 2\n" + + " \"fetchThreads\": 2,\n" + + " \"appendableIndexSpec\": { \"type\" : \"onheap\" }\n" + "}"; KinesisIndexTaskTuningConfig config = (KinesisIndexTaskTuningConfig) mapper.readValue( @@ -116,6 +119,7 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); @@ -169,6 +173,7 @@ public void testSerdeWithModifiedTuningConfigAddedField() throws IOException mapper.readValue(serialized, TestModifiedKinesisIndexTaskTuningConfig.class); Assert.assertEquals(null, deserialized.getExtra()); + Assert.assertEquals(base.getAppendableIndexSpec(), deserialized.getAppendableIndexSpec()); Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); @@ -228,6 +233,7 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException KinesisIndexTaskTuningConfig deserialized = mapper.readValue(serialized, KinesisIndexTaskTuningConfig.class); + Assert.assertEquals(base.getAppendableIndexSpec(), deserialized.getAppendableIndexSpec()); Assert.assertEquals(base.getMaxRowsInMemory(), deserialized.getMaxRowsInMemory()); Assert.assertEquals(base.getMaxBytesInMemory(), deserialized.getMaxBytesInMemory()); Assert.assertEquals(base.getMaxRowsPerSegment(), deserialized.getMaxRowsPerSegment()); @@ -320,6 +326,7 @@ public void testConvert() ); KinesisIndexTaskTuningConfig copy = (KinesisIndexTaskTuningConfig) original.convertToTaskTuningConfig(); + Assert.assertEquals(original.getAppendableIndexSpec(), copy.getAppendableIndexSpec()); Assert.assertEquals(1, copy.getMaxRowsInMemory()); Assert.assertEquals(3, copy.getMaxBytesInMemory()); Assert.assertEquals(2, copy.getMaxRowsPerSegment().intValue()); diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java index d50321043ce8..ff3edc46e429 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTuningConfigTest.java @@ -24,6 +24,7 @@ import org.apache.druid.indexing.kinesis.KinesisIndexingServiceModule; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.TuningConfig; import org.joda.time.Duration; import org.joda.time.Period; @@ -58,6 +59,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(1000000, config.getMaxRowsInMemory()); Assert.assertEquals(5_000_000, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT10M"), config.getIntermediatePersistPeriod()); @@ -92,7 +94,8 @@ public void testSerdeWithNonDefaults() throws Exception + " \"chatRetries\": 14,\n" + " \"httpTimeout\": \"PT15S\",\n" + " \"shutdownTimeout\": \"PT95S\",\n" - + " \"repartitionTransitionDuration\": \"PT500S\"\n" + + " \"repartitionTransitionDuration\": \"PT500S\",\n" + + " \"appendableIndexSpec\": { \"type\" : \"onheap\" }\n" + "}"; KinesisSupervisorTuningConfig config = (KinesisSupervisorTuningConfig) mapper.readValue( @@ -106,6 +109,7 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals(new File("/tmp/xxx"), config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(100, config.getMaxRowsInMemory()); Assert.assertEquals(100, config.getMaxRowsPerSegment().intValue()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java index 1397c0f8afe9..fce828b64be1 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopTuningConfigTest.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.junit.Assert; import org.junit.Test; @@ -69,6 +70,7 @@ public void testSerde() throws Exception Assert.assertEquals("/tmp/workingpath", actual.getWorkingPath()); Assert.assertEquals("version", actual.getVersion()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), actual.getAppendableIndexSpec()); Assert.assertNotNull(actual.getPartitionsSpec()); Assert.assertEquals(ImmutableMap.>of(), actual.getShardSpecs()); Assert.assertEquals(new IndexSpec(), actual.getIndexSpec()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index ebfb304cf0d2..9a1240af836e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1529,7 +1529,8 @@ public boolean equals(Object o) return false; } IndexTuningConfig that = (IndexTuningConfig) o; - return maxRowsInMemory == that.maxRowsInMemory && + return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && + maxRowsInMemory == that.maxRowsInMemory && maxBytesInMemory == that.maxBytesInMemory && maxPendingPersists == that.maxPendingPersists && forceGuaranteedRollup == that.forceGuaranteedRollup && diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index ff2a52f57eea..77128fa13973 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -292,7 +292,8 @@ public boolean equals(Object o) return false; } SeekableStreamIndexTaskTuningConfig that = (SeekableStreamIndexTaskTuningConfig) o; - return maxRowsInMemory == that.maxRowsInMemory && + return Objects.equals(appendableIndexSpec, that.appendableIndexSpec) && + maxRowsInMemory == that.maxRowsInMemory && maxBytesInMemory == that.maxBytesInMemory && maxPendingPersists == that.maxPendingPersists && reportParseExceptions == that.reportParseExceptions && 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 889307993081..655c8326c830 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 @@ -470,5 +470,11 @@ public long getDefaultMaxBytesInMemory() // we halved default to 1/6(max jvm memory) return JvmUtils.getRuntimeInfo().getMaxHeapSizeBytes() / 6; } + + @Override + public boolean equals(Object that) + { + return that.getClass().equals(this.getClass()); + } } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java deleted file mode 100644 index 60cac8631d6c..000000000000 --- a/processing/src/test/java/org/apache/druid/segment/incremental/AppendableIndexSpecTest.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Test; - -/** - */ -public class AppendableIndexSpecTest -{ - private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); - - @Test - public void testSerializationOnheapString() throws Exception - { - AppendableIndexSpec spec = JSON_MAPPER.readValue( - "{\"type\": \"onheap\"}", AppendableIndexSpec.class - ); - Assert.assertTrue(spec instanceof OnheapIncrementalIndex.Spec); - } - - @Test - public void testSerializationDefaultString() throws Exception - { - AppendableIndexSpec spec = JSON_MAPPER.readValue( - "{}", AppendableIndexSpec.class - ); - Assert.assertTrue(spec instanceof OnheapIncrementalIndex.Spec); - } -} diff --git a/server/src/test/java/org/apache/druid/segment/indexing/RealtimeTuningConfigTest.java b/server/src/test/java/org/apache/druid/segment/indexing/RealtimeTuningConfigTest.java index ea3f7cf3f7bd..72f0b8bbfe91 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/RealtimeTuningConfigTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/RealtimeTuningConfigTest.java @@ -23,6 +23,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.hamcrest.CoreMatchers; import org.joda.time.Period; @@ -89,6 +90,7 @@ public void testSerdeWithDefaults() throws Exception ); Assert.assertNotNull(config.getBasePersistDirectory()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(0, config.getHandoffConditionTimeout()); Assert.assertEquals(0, config.getAlertTimeout()); Assert.assertEquals(new IndexSpec(), config.getIndexSpec()); @@ -119,7 +121,8 @@ public void testSerdeWithNonDefaults() throws Exception + " \"handoffConditionTimeout\": 100,\n" + " \"alertTimeout\": 70,\n" + " \"indexSpec\": { \"metricCompression\" : \"NONE\" },\n" - + " \"indexSpecForIntermediatePersists\": { \"dimensionCompression\" : \"uncompressed\" }\n" + + " \"indexSpecForIntermediatePersists\": { \"dimensionCompression\" : \"uncompressed\" },\n" + + " \"appendableIndexSpec\": { \"type\" : \"onheap\" }\n" + "}"; ObjectMapper mapper = TestHelper.makeJsonMapper(); @@ -134,6 +137,7 @@ public void testSerdeWithNonDefaults() throws Exception ); Assert.assertEquals("/tmp/xxx", config.getBasePersistDirectory().toString()); + Assert.assertEquals(new OnheapIncrementalIndex.Spec(), config.getAppendableIndexSpec()); Assert.assertEquals(100, config.getHandoffConditionTimeout()); Assert.assertEquals(70, config.getAlertTimeout()); Assert.assertEquals(new Period("PT1H"), config.getIntermediatePersistPeriod()); From 9762a81e9f78e94c9a6fc7ec586f7f7b12a5226c Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 8 Oct 2020 14:20:18 +0300 Subject: [PATCH 15/20] Modify hashCode() to conform with equals() --- .../org/apache/druid/indexing/common/task/IndexTask.java | 1 + .../seekablestream/SeekableStreamIndexTaskTuningConfig.java | 1 + .../druid/segment/incremental/OnheapIncrementalIndex.java | 6 ++++++ 3 files changed, 8 insertions(+) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 9a1240af836e..78be6537ca70 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1550,6 +1550,7 @@ public boolean equals(Object o) public int hashCode() { return Objects.hash( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, partitionsSpec, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index 77128fa13973..9d8cf7df216a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -316,6 +316,7 @@ public boolean equals(Object o) public int hashCode() { return Objects.hash( + appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, partitionsSpec, 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 655c8326c830..a72124a21533 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 @@ -476,5 +476,11 @@ public boolean equals(Object that) { return that.getClass().equals(this.getClass()); } + + @Override + public int hashCode() + { + return Objects.hash(this.getClass()); + } } } From 829b86f9548952795c4f3808679adfacb3f5b97a Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 8 Oct 2020 19:01:12 +0300 Subject: [PATCH 16/20] Add comment where building incremental-index --- .../main/java/org/apache/druid/indexer/IndexGeneratorJob.java | 1 + .../java/org/apache/druid/segment/realtime/plumber/Sink.java | 2 ++ 2 files changed, 3 insertions(+) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index 9ce140ee2a4b..abc0b3bc96f5 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -301,6 +301,7 @@ private static IncrementalIndex makeIncrementalIndex( .withRollup(config.getSchema().getDataSchema().getGranularitySpec().isRollup()) .build(); + // Build the incremental-index according to the spec that was chosen by the user IncrementalIndex newIndex = tuningConfig.getAppendableIndexSpec().builder() .setIndexSchema(indexSchema) .setMaxRowCount(tuningConfig.getRowFlushBoundary()) diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index ff20c1ee81bf..ccf5e4c5421b 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -328,6 +328,8 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) .withMetrics(schema.getAggregators()) .withRollup(schema.getGranularitySpec().isRollup()) .build(); + + // Build the incremental-index according to the spec that was chosen by the user final IncrementalIndex newIndex = appendableIndexSpec.builder() .setIndexSchema(indexSchema) .setMaxRowCount(maxRowsInMemory) From afac4d882e6c0d3159163ff62e0333f01b491a52 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 15 Oct 2020 18:05:51 +0300 Subject: [PATCH 17/20] Add "EqualsVerifier" tests --- extensions-core/kafka-indexing-service/pom.xml | 5 +++++ .../indexing/kafka/KafkaIndexTaskTuningConfigTest.java | 9 +++++++++ extensions-core/kinesis-indexing-service/pom.xml | 5 +++++ .../kinesis/KinesisIndexTaskTuningConfigTest.java | 9 +++++++++ .../apache/druid/indexing/common/task/IndexTaskTest.java | 9 +++++++++ .../batch/parallel/ParallelIndexTuningConfigTest.java | 9 +++++++++ 6 files changed, 46 insertions(+) diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index d192b6876d2f..7cc1ebf65bf0 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -203,6 +203,11 @@ assertj-core test + + nl.jqno.equalsverifier + equalsverifier + test + diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java index 9d38430acc49..8888a69a0d89 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTuningConfigTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorTuningConfig; import org.apache.druid.indexing.kafka.test.TestModifiedKafkaIndexTaskTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; @@ -259,4 +260,12 @@ public void testSerdeWithModifiedTuningConfigRemovedField() throws IOException Assert.assertEquals(base.getMaxParseExceptions(), deserialized.getMaxParseExceptions()); Assert.assertEquals(base.getMaxSavedParseExceptions(), deserialized.getMaxSavedParseExceptions()); } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(KafkaIndexTaskTuningConfig.class) + .usingGetClass() + .verify(); + } } diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index 6b08d50ea62d..0659bedc0f13 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -192,6 +192,11 @@ system-rules test + + nl.jqno.equalsverifier + equalsverifier + test + diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java index 7a255966e41b..69a70a494f16 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTuningConfigTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorTuningConfig; import org.apache.druid.indexing.kinesis.test.TestModifiedKinesisIndexTaskTuningConfig; import org.apache.druid.jackson.DefaultObjectMapper; @@ -348,4 +349,12 @@ public void testConvert() Assert.assertEquals(100, copy.getMaxRecordsPerPoll()); Assert.assertEquals(new Period().withDays(Integer.MAX_VALUE), copy.getIntermediateHandoffPeriod()); } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(KinesisIndexTaskTuningConfig.class) + .usingGetClass() + .verify(); + } } 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 be16a953dbdd..0d4810acb47c 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 @@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; import com.google.common.io.Files; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.CSVParseSpec; import org.apache.druid.data.input.impl.CsvInputFormat; @@ -2002,4 +2003,12 @@ private static IndexIngestionSpec createIngestionSpec( ); } } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(IndexTuningConfig.class) + .usingGetClass() + .verify(); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java index 13ba39b93a65..a826ae9cd42d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfigTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; @@ -358,4 +359,12 @@ public void testConstructorWithDynamicPartitionsSpecAndForceGuaranteedRollupFail null ); } + + @Test + public void testEqualsAndHashCode() + { + EqualsVerifier.forClass(ParallelIndexTuningConfig.class) + .usingGetClass() + .verify(); + } } From 5c8c18f69ca287db043ceee8aed04309c1cdf6ce Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Thu, 15 Oct 2020 23:13:16 +0300 Subject: [PATCH 18/20] Revert some of the API back to AppenderatorConfig --- .../java/org/apache/druid/indexer/HadoopTuningConfig.java | 3 --- .../org/apache/druid/segment/indexing/TuningConfig.java | 8 -------- .../segment/realtime/appenderator/AppenderatorConfig.java | 8 ++++++++ 3 files changed, 8 insertions(+), 11 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index bdfe98334e43..8c3e6fe326df 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -193,7 +193,6 @@ public String getVersion() } @JsonProperty - @Override public DimensionBasedPartitionsSpec getPartitionsSpec() { return partitionsSpec; @@ -206,14 +205,12 @@ public Map> getShardSpecs() } @JsonProperty - @Override public IndexSpec getIndexSpec() { return indexSpec; } @JsonProperty - @Override public IndexSpec getIndexSpecForIntermediatePersists() { return indexSpecForIntermediatePersists; diff --git a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java index e659c3973e43..e3a4e1fadd05 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/TuningConfig.java @@ -21,8 +21,6 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.incremental.AppendableIndexSpec; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; @@ -68,10 +66,4 @@ default long getMaxBytesInMemoryOrDefault() return Long.MAX_VALUE; } } - - PartitionsSpec getPartitionsSpec(); - - IndexSpec getIndexSpec(); - - IndexSpec getIndexSpecForIntermediatePersists(); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java index ec760b22c22e..259e58c400a6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -19,6 +19,8 @@ package org.apache.druid.segment.realtime.appenderator; +import org.apache.druid.indexer.partitions.PartitionsSpec; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; @@ -55,11 +57,17 @@ default Long getMaxTotalRows() throw new UnsupportedOperationException("maxTotalRows is not implemented."); } + PartitionsSpec getPartitionsSpec(); + /** * Period that sets frequency to persist to local storage if no other thresholds are met */ Period getIntermediatePersistPeriod(); + IndexSpec getIndexSpec(); + + IndexSpec getIndexSpecForIntermediatePersists(); + File getBasePersistDirectory(); AppenderatorConfig withBasePersistDirectory(File basePersistDirectory); From b04ae43b8bb83af05a54d72cfe39ef788f8e3d61 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Wed, 21 Oct 2020 14:05:18 +0300 Subject: [PATCH 19/20] Don't use multi-line comments --- .../java/org/apache/druid/indexer/HadoopTuningConfig.java | 4 ++-- .../common/index/RealtimeAppenderatorTuningConfig.java | 4 ++-- .../java/org/apache/druid/indexing/common/task/IndexTask.java | 4 ++-- .../seekablestream/SeekableStreamIndexTaskTuningConfig.java | 4 ++-- .../apache/druid/segment/indexing/RealtimeTuningConfig.java | 4 ++-- 5 files changed, 10 insertions(+), 10 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java index 8c3e6fe326df..f1a8cc8adca0 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopTuningConfig.java @@ -145,8 +145,8 @@ public HadoopTuningConfig( ? DEFAULT_ROW_FLUSH_BOUNDARY : maxRowsInMemoryCOMPAT : maxRowsInMemory; this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; - /** initializing this to 0, it will be lazily initialized to a value - * @see #getMaxBytesInMemoryOrDefault() */ + // initializing this to 0, it will be lazily initialized to a value + // @see #getMaxBytesInMemoryOrDefault() this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.leaveIntermediate = leaveIntermediate; this.cleanupOnFailure = cleanupOnFailure == null ? true : cleanupOnFailure; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java index 5daf6cf03407..1538e1581929 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/RealtimeAppenderatorTuningConfig.java @@ -97,8 +97,8 @@ public RealtimeAppenderatorTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - /** initializing this to 0, it will be lazily initialized to a value - * @see #getMaxBytesInMemoryOrDefault() */ + // initializing this to 0, it will be lazily initialized to a value + // @see #getMaxBytesInMemoryOrDefault() this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); this.intermediatePersistPeriod = intermediatePersistPeriod == null diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 78be6537ca70..f351a5db6c7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1269,8 +1269,8 @@ private IndexTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? TuningConfig.DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - /** initializing this to 0, it will be lazily initialized to a value - * @see #getMaxBytesInMemoryOrDefault() */ + // initializing this to 0, it will be lazily initialized to a value + // @see #getMaxBytesInMemoryOrDefault() this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.partitionsSpec = partitionsSpec; this.indexSpec = indexSpec == null ? DEFAULT_INDEX_SPEC : indexSpec; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java index 9d8cf7df216a..b56d8e5f2ed6 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTuningConfig.java @@ -90,8 +90,8 @@ public SeekableStreamIndexTaskTuningConfig( this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? defaults.getMaxRowsInMemory() : maxRowsInMemory; this.partitionsSpec = new DynamicPartitionsSpec(maxRowsPerSegment, maxTotalRows); - /** initializing this to 0, it will be lazily initialized to a value - * @see #getMaxBytesInMemoryOrDefault() */ + // initializing this to 0, it will be lazily initialized to a value + // @see #getMaxBytesInMemoryOrDefault() this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? defaults.getIntermediatePersistPeriod() diff --git a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java index 44dbe82b749a..48fb592d9431 100644 --- a/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/org/apache/druid/segment/indexing/RealtimeTuningConfig.java @@ -137,8 +137,8 @@ public RealtimeTuningConfig( { this.appendableIndexSpec = appendableIndexSpec == null ? DEFAULT_APPENDABLE_INDEX : appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory == null ? DEFAULT_MAX_ROWS_IN_MEMORY : maxRowsInMemory; - /** initializing this to 0, it will be lazily initialized to a value - * @see #getMaxBytesInMemoryOrDefault() */ + // initializing this to 0, it will be lazily initialized to a value + // @see #getMaxBytesInMemoryOrDefault() this.maxBytesInMemory = maxBytesInMemory == null ? 0 : maxBytesInMemory; this.intermediatePersistPeriod = intermediatePersistPeriod == null ? DEFAULT_INTERMEDIATE_PERSIST_PERIOD From 7de9b493601541b8f1a2e76ddb5bdeb6d7cfb0e8 Mon Sep 17 00:00:00 2001 From: Liran Funaro Date: Wed, 21 Oct 2020 14:11:05 +0300 Subject: [PATCH 20/20] Remove knob documentation (deferred) --- docs/ingestion/index.md | 12 ------------ website/.spelling | 1 - 2 files changed, 13 deletions(-) diff --git a/docs/ingestion/index.md b/docs/ingestion/index.md index fe8277afa9d6..0925675f9200 100644 --- a/docs/ingestion/index.md +++ b/docs/ingestion/index.md @@ -712,9 +712,6 @@ is: "metricCompression": "lz4", "longEncoding": "longs" }, - "appendableIndexSpec": { - "type": "onheap" - }, } ``` @@ -725,7 +722,6 @@ is: |maxRowsInMemory|The maximum number of records to store in memory before persisting to disk. Note that this is the number of rows post-rollup, and so it may not be equal to the number of input records. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).|`1000000`| |maxBytesInMemory|The maximum aggregate size of records, in bytes, to store in the JVM heap before persisting. This is based on a rough estimate of memory usage. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).

Setting maxBytesInMemory to -1 disables this check, meaning Druid will rely entirely on maxRowsInMemory to control memory usage. Setting it to zero means the default value will be used (one-sixth of JVM heap size).

Note that the estimate of memory usage is designed to be an overestimate, and can be especially high when using complex ingest-time aggregators, including sketches. If this causes your indexing workloads to persist to disk too often, you can set maxBytesInMemory to -1 and rely on maxRowsInMemory instead.|One-sixth of max JVM heap size| |indexSpec|Tune how data is indexed. See below for more information.|See table below| -|appendableIndexSpec|Tune in-memory index. See below for more information.|See table below| |Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../development/extensions-core/kafka-ingestion.md#tuningconfig), [Kinesis indexing service](../development/extensions-core/kinesis-ingestion.md#tuningconfig), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).|| #### `indexSpec` @@ -741,11 +737,3 @@ The `indexSpec` object can include the following properties: Beyond these properties, each ingestion method has its own specific tuning properties. See the documentation for each [ingestion method](#ingestion-methods) for details. - -#### `appendableIndexSpec` - -|Field|Description|Default| -|-----|-----------|-------| -|type|Each in-memory index has its own tuning type code. You must specify the type code that matches your in-memory index. Common options are `onheap`, and `offheap`.|`onheap`| - -Beyond these properties, each in-memory index has its own specific tuning properties. diff --git a/website/.spelling b/website/.spelling index 96d5d5cc4390..a783d32ddc3b 100644 --- a/website/.spelling +++ b/website/.spelling @@ -1772,7 +1772,6 @@ missingValue schemaless spatialDimensions useFieldDiscovery -appendableIndexSpec - ../docs/tutorials/index.md 4CPU cityName