From 348f17eb388b24644e2775201a416df25cebec19 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 11 Jan 2022 22:33:08 +0530 Subject: [PATCH 01/15] Improve mem estimation in Aggregator and DimensionIndexer --- .../datasketches/theta/SketchAggregator.java | 69 +++++++++++++ .../theta/SketchAggregatorFactory.java | 9 ++ .../theta/SketchAggregatorFactoryTest.java | 19 ++++ .../indexing/common/task/AbstractTask.java | 11 +++ .../AppenderatorDriverRealtimeIndexTask.java | 3 +- .../common/task/BatchAppenderators.java | 18 ++-- .../druid/indexing/common/task/IndexTask.java | 3 +- .../druid/indexing/common/task/Tasks.java | 2 + .../parallel/PartialSegmentGenerateTask.java | 8 +- .../batch/parallel/SinglePhaseSubTask.java | 7 +- .../SeekableStreamIndexTask.java | 3 +- .../common/task/AppenderatorsTest.java | 9 +- .../common/task/BatchAppenderatorsTest.java | 9 +- .../common/task/TestAppenderatorsManager.java | 24 +++-- .../druid/query/aggregation/Aggregator.java | 8 +- .../query/aggregation/AggregatorAndSize.java | 49 ++++++++++ .../query/aggregation/AggregatorFactory.java | 9 ++ .../druid/segment/DimensionDictionary.java | 25 +++++ .../druid/segment/DimensionIndexer.java | 2 +- .../druid/segment/DoubleDimensionIndexer.java | 4 +- .../druid/segment/EncodedDimensionValue.java | 48 +++++++++ .../druid/segment/FloatDimensionIndexer.java | 4 +- .../druid/segment/LongDimensionIndexer.java | 4 +- .../druid/segment/StringDimensionIndexer.java | 8 +- .../incremental/AppendableIndexBuilder.java | 7 ++ .../segment/incremental/IncrementalIndex.java | 33 +++++-- .../incremental/IncrementalIndexRow.java | 20 ++-- .../incremental/OnheapIncrementalIndex.java | 97 ++++++++++++------- .../OnheapIncrementalIndexBenchmark.java | 6 +- .../appenderator/AppenderatorImpl.java | 7 +- .../realtime/appenderator/Appenderators.java | 24 +++-- .../appenderator/AppenderatorsManager.java | 12 ++- .../appenderator/BatchAppenderator.java | 7 +- .../DefaultOfflineAppenderatorFactory.java | 3 +- .../DefaultRealtimeAppenderatorFactory.java | 3 +- ...DummyForInjectionAppenderatorsManager.java | 12 ++- .../PeonAppenderatorsManager.java | 24 +++-- .../appenderator/StreamAppenderator.java | 7 +- .../UnifiedIndexerAppenderatorsManager.java | 24 +++-- .../realtime/plumber/RealtimePlumber.java | 2 + .../druid/segment/realtime/plumber/Sink.java | 8 +- ...edSegmensSinksBatchAppenderatorTester.java | 3 +- ...enAndClosedSegmentsAppenderatorTester.java | 6 +- .../StreamAppenderatorTester.java | 3 +- ...nifiedIndexerAppenderatorsManagerTest.java | 3 +- .../plumber/RealtimePlumberSchoolTest.java | 3 + .../segment/realtime/plumber/SinkTest.java | 2 + 47 files changed, 536 insertions(+), 135 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java create mode 100644 processing/src/main/java/org/apache/druid/segment/EncodedDimensionValue.java diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java index b53d89d41d2a..6902fc2f8826 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -21,6 +21,7 @@ import org.apache.datasketches.Family; import org.apache.datasketches.theta.SetOperation; +import org.apache.datasketches.theta.Sketch; import org.apache.datasketches.theta.Union; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; @@ -28,16 +29,34 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import javax.annotation.Nullable; +import java.lang.reflect.Field; import java.util.List; public class SketchAggregator implements Aggregator { + private final BaseObjectColumnValueSelector selector; private final int size; @Nullable private Union union; + @Nullable + private Sketch sketch; + + @Nullable + private static final Field sketchField; + static { + try { + sketchField = Class.forName("org.apache.datasketches.theta.UnionImpl") + .getDeclaredField("gadget_"); + sketchField.setAccessible(true); + } + catch (NoSuchFieldException | ClassNotFoundException e) { + throw new ISE("Could not initialize SketchAggregator", e); + } + } + public SketchAggregator(BaseObjectColumnValueSelector selector, int size) { this.selector = selector; @@ -49,6 +68,16 @@ private void initUnion() union = (Union) SetOperation.builder().setNominalEntries(size).build(Family.UNION); } + private void initSketch() + { + try { + sketch = (Sketch) sketchField.get(union); + } + catch (IllegalAccessException e) { + throw new ISE("Could not initialize sketch field in SketchAggregator", e); + } + } + @Override public void aggregate() { @@ -64,6 +93,36 @@ public void aggregate() } } + @Override + public long aggregateWithSize() + { + Object update = selector.getObject(); + if (update == null) { + return 0; + } + synchronized (this) { + long unionSizeDelta = 0; + if (union == null) { + initUnion(); + + // Fields in UnionImpl: a sketch reference, a short, a long and a boolean + unionSizeDelta = Long.BYTES + Short.BYTES + Long.BYTES + 1; + } + + long initialSketchSize = 0; + if (sketch == null) { + initSketch(); + } else { + initialSketchSize = sketch.getCurrentBytes(); + } + + updateUnion(union, update); + + long sketchSizeDelta = sketch.getCurrentBytes() - initialSketchSize; + return sketchSizeDelta + unionSizeDelta; + } + } + @Override public Object get() { @@ -133,4 +192,14 @@ static void updateUnion(Union union, Object update) throw new ISE("Illegal type received while theta sketch merging [%s]", update.getClass()); } } + + /** + * Gets the initial size of this aggregator in bytes. + */ + public long getInitialSizeBytes() + { + // SketchAggregator has 3 references and an int + return 3 * Long.BYTES + Integer.BYTES; + } + } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java index a57c547cd1b8..64a7fd5014a9 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java @@ -32,6 +32,7 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.ObjectAggregateCombiner; +import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnInspector; @@ -80,6 +81,14 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) return new SketchAggregator(selector, size); } + @Override + public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) + { + BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); + final SketchAggregator aggregator = new SketchAggregator(selector, size); + return new AggregatorAndSize(aggregator, aggregator.getInitialSizeBytes()); + } + @SuppressWarnings("unchecked") @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java index 61efd5fb6656..b5d3a1a2dd2d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java @@ -23,14 +23,18 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.datasketches.theta.oldapi.OldSketchBuildAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.theta.oldapi.OldSketchMergeAggregatorFactory; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -61,6 +65,21 @@ public void testMaxIntermediateSize() Assert.assertEquals(524320, AGGREGATOR_32768.getMaxIntermediateSize()); } + @Test + public void testFactorizeSized() + { + ColumnSelectorFactory colSelectorFactory = EasyMock.mock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector(EasyMock.anyString())) + .andReturn(EasyMock.createMock(ColumnValueSelector.class)).anyTimes(); + EasyMock.replay(colSelectorFactory); + + AggregatorAndSize aggregatorAndSize = AGGREGATOR_16384.factorizeWithSize(colSelectorFactory); + Assert.assertEquals(591, aggregatorAndSize.getInitialSizeBytes()); + + aggregatorAndSize = AGGREGATOR_32768.factorizeWithSize(colSelectorFactory); + Assert.assertEquals(1103, aggregatorAndSize.getInitialSizeBytes()); + } + @Test public void testResultArraySignature() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java index 8964a1e1c684..23adeb6ff81a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java @@ -208,4 +208,15 @@ public Map getContext() { return context; } + + /** + * Whether maximum memory usage should be considered in estimation for indexing tasks. + */ + protected boolean isUseMaxMemoryEstimates() + { + return getContextValue( + Tasks.USE_MAX_MEMORY_ESTIMATES, + Tasks.DEFAULT_USE_MAX_MEMORY_ESTIMATES + ); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 1ef23b7b3e6a..7eb60b27ccbd 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -780,7 +780,8 @@ private Appenderator newAppenderator( toolbox.getCacheConfig(), toolbox.getCachePopulatorStats(), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + isUseMaxMemoryEstimates() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java index 8d15dc815e1f..a545f5190007 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/BatchAppenderators.java @@ -44,7 +44,8 @@ public static Appenderator newAppenderator( DataSchema dataSchema, AppenderatorConfig appenderatorConfig, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { return newAppenderator( @@ -56,7 +57,8 @@ public static Appenderator newAppenderator( appenderatorConfig, toolbox.getSegmentPusher(), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } @@ -69,7 +71,8 @@ public static Appenderator newAppenderator( AppenderatorConfig appenderatorConfig, DataSegmentPusher segmentPusher, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.OPEN_SEGMENTS) { @@ -83,7 +86,8 @@ public static Appenderator newAppenderator( toolbox.getIndexIO(), toolbox.getIndexMergerV9(), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS) { return appenderatorsManager.createClosedSegmentsOfflineAppenderatorForTask( @@ -96,7 +100,8 @@ public static Appenderator newAppenderator( toolbox.getIndexIO(), toolbox.getIndexMergerV9(), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } else if (toolbox.getConfig().getBatchProcessingMode() == TaskConfig.BatchProcessingMode.CLOSED_SEGMENTS_SINKS) { return appenderatorsManager.createOfflineAppenderatorForTask( @@ -109,7 +114,8 @@ public static Appenderator newAppenderator( toolbox.getIndexIO(), toolbox.getIndexMergerV9(), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } else { throw new IAE("Invalid batchProcesingMode[%s]", toolbox.getConfig().getBatchProcessingMode()); 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 e1b6fd88b276..fc35f0b9cb44 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 @@ -905,7 +905,8 @@ private TaskStatus generateAndPublishSegments( dataSchema, tuningConfig, buildSegmentsMeters, - buildSegmentsParseExceptionHandler + buildSegmentsParseExceptionHandler, + isUseMaxMemoryEstimates() ); boolean exceptionOccurred = false; try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index 045cee3feb17..55e46105d975 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -46,11 +46,13 @@ public class Tasks public static final long DEFAULT_LOCK_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); public static final boolean DEFAULT_FORCE_TIME_CHUNK_LOCK = true; public static final boolean DEFAULT_STORE_COMPACTION_STATE = false; + public static final boolean DEFAULT_USE_MAX_MEMORY_ESTIMATES = true; public static final String PRIORITY_KEY = "priority"; public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout"; public static final String FORCE_TIME_CHUNK_LOCK_KEY = "forceTimeChunkLock"; public static final String USE_SHARED_LOCK = "useSharedLock"; + public static final String USE_MAX_MEMORY_ESTIMATES = "useMaxMemoryEstimates"; /** * This context is used in compaction. When it is set in the context, the segments created by the task diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index a69827cfb6b3..2f7f67c7b2bb 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -30,6 +30,7 @@ import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SequenceNameFunction; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.batch.parallel.iterator.IndexTaskInputRowIteratorBuilder; import org.apache.druid.indexing.worker.shuffle.ShuffleDataSegmentPusher; import org.apache.druid.query.DruidMetrics; @@ -169,6 +170,10 @@ private List generateSegments( tuningConfig.getMaxParseExceptions(), tuningConfig.getMaxSavedParseExceptions() ); + final boolean useMaxMemoryEstimates = getContextValue( + Tasks.USE_MAX_MEMORY_ESTIMATES, + Tasks.DEFAULT_USE_MAX_MEMORY_ESTIMATES + ); final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), toolbox.getAppenderatorsManager(), @@ -178,7 +183,8 @@ private List generateSegments( tuningConfig, new ShuffleDataSegmentPusher(supervisorTaskId, getId(), toolbox.getIntermediaryDataManager()), buildSegmentsMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); boolean exceptionOccurred = false; try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 0a9f154f1480..2970c509b7c7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -393,6 +393,10 @@ private Set generateAndPushSegments( useLineageBasedSegmentAllocation ); + final boolean useMaxMemoryEstimates = getContextValue( + Tasks.USE_MAX_MEMORY_ESTIMATES, + Tasks.DEFAULT_USE_MAX_MEMORY_ESTIMATES + ); final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), toolbox.getAppenderatorsManager(), @@ -401,7 +405,8 @@ private Set generateAndPushSegments( dataSchema, tuningConfig, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); boolean exceptionOccurred = false; try ( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 4fac734f0232..b74569f7f3a7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -201,7 +201,8 @@ public Appenderator newAppenderator( toolbox.getCacheConfig(), toolbox.getCachePopulatorStats(), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + isUseMaxMemoryEstimates() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java index b90b95df0d0a..72e3adafdd07 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorsTest.java @@ -245,7 +245,8 @@ public Map makeLoadSpec(URI uri) indexIO, indexMerger, rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + true ); break; case "CLOSED_SEGMENTS": @@ -259,7 +260,8 @@ public Map makeLoadSpec(URI uri) indexIO, indexMerger, rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + true ); break; @@ -274,7 +276,8 @@ public Map makeLoadSpec(URI uri) indexIO, indexMerger, rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + true ); break; default: diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java index 6813dcc97c4e..ada1363a9280 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/BatchAppenderatorsTest.java @@ -257,7 +257,8 @@ public Map makeLoadSpec(URI uri) false, Integer.MAX_VALUE, 0 - ) + ), + true ); break; case "CLOSED_SEGMENTS": @@ -279,7 +280,8 @@ public Map makeLoadSpec(URI uri) false, Integer.MAX_VALUE, 0 - ) + ), + true ); break; @@ -302,7 +304,8 @@ public Map makeLoadSpec(URI uri) false, Integer.MAX_VALUE, 0 - ) + ), + true ); break; default: diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java index bcdbb66226c2..0fc794337720 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TestAppenderatorsManager.java @@ -67,7 +67,8 @@ public Appenderator createRealtimeAppenderatorForTask( CacheConfig cacheConfig, CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { realtimeAppenderator = Appenderators.createRealtime( @@ -88,7 +89,8 @@ public Appenderator createRealtimeAppenderatorForTask( cacheConfig, cachePopulatorStats, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + true ); return realtimeAppenderator; } @@ -104,7 +106,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { return Appenderators.createOpenSegmentsOffline( @@ -117,7 +120,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + true ); } @@ -132,7 +136,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { return Appenderators.createClosedSegmentsOffline( @@ -145,7 +150,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } @@ -160,7 +166,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { return Appenderators.createOffline( @@ -173,7 +180,8 @@ public Appenderator createOfflineAppenderatorForTask( indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + true ); } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java index fe05e4759dca..c4b225a02569 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java @@ -42,6 +42,12 @@ public interface Aggregator extends Closeable { void aggregate(); + default long aggregateWithSize() + { + aggregate(); + return 0; + } + @Nullable Object get(); float getFloat(); @@ -54,7 +60,7 @@ public interface Aggregator extends Closeable */ default double getDouble() { - return (double) getFloat(); + return getFloat(); } /** diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java new file mode 100644 index 000000000000..6259c7a75a4f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java @@ -0,0 +1,49 @@ +/* + * 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.query.aggregation; + +/** + * Encapsulates an {@link Aggregator} and the initial size in bytes required by + * the Aggregator. + */ +public class AggregatorAndSize +{ + + // TODO: include default overhead for object sizes + + private final Aggregator aggregator; + private final long initialSizeBytes; + + public AggregatorAndSize(Aggregator aggregator, long initialSizeBytes) + { + this.aggregator = aggregator; + this.initialSizeBytes = initialSizeBytes; + } + + public Aggregator getAggregator() + { + return aggregator; + } + + public long getInitialSizeBytes() + { + return initialSizeBytes; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 45f9328532cb..a65f1e02d897 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -69,6 +69,15 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFact throw new UOE("Aggregator[%s] cannot vectorize", getClass().getName()); } + /** + * Creates an {@link Aggregator} based on the provided column selector factory. + * + * @return AggregatorAndSize which contains the actual aggregator and its initial size. + */ + public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) { + return new AggregatorAndSize(factorize(metricFactory), getMaxIntermediateSize()); + } + /** * Returns whether or not this aggregation class supports vectorization. The default implementation returns false. */ diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java index 345cd7ba7737..0cf03fca57f9 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java @@ -22,9 +22,11 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantReadWriteLock; /** @@ -44,6 +46,7 @@ public class DimensionDictionary> private T maxValue = null; private volatile int idForNull = ABSENT_VALUE_ID; + private final AtomicLong sizeInBytes = new AtomicLong(0L); private final Object2IntMap valueToId = new Object2IntOpenHashMap<>(); private final List idToValue = new ArrayList<>(); @@ -96,6 +99,11 @@ public int size() } } + public long sizeInBytes() + { + return sizeInBytes.get(); + } + public int add(@Nullable T originalValue) { lock.writeLock().lock(); @@ -114,6 +122,11 @@ public int add(@Nullable T originalValue) final int index = idToValue.size(); valueToId.put(originalValue, index); idToValue.add(originalValue); + + long sizeOfString = getObjectSize(originalValue); + long sizeOfReference = Long.BYTES; + sizeInBytes.addAndGet(sizeOfString + 2 * sizeOfReference); + minValue = minValue == null || minValue.compareTo(originalValue) > 0 ? originalValue : minValue; maxValue = maxValue == null || maxValue.compareTo(originalValue) < 0 ? originalValue : maxValue; return index; @@ -160,4 +173,16 @@ public SortedDimensionDictionary sort() lock.readLock().unlock(); } } + + private long getObjectSize(@Nonnull T object) + { + // According to https://www.ibm.com/developerworks/java/library/j-codetoheap/index.html + // String has the following memory usuage... + // 28 bytes of data for String metadata (class pointer, flags, locks, hash, count, offset, reference to char array) + // 16 bytes of data for the char array metadata (class pointer, flags, locks, size) + // 2 bytes for every letter of the string + String val = object.toString(); + return 28 + 16 + (2L * val.length()); + } + } diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java index 00ae11da956f..e94726ce6fab 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java @@ -127,7 +127,7 @@ public interface DimensionIndexer * @return An array containing an encoded representation of the input row value. */ @Nullable - EncodedKeyComponentType processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions); + EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions); /** * This method will be called while building an {@link IncrementalIndex} whenever a known dimension column (either diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java index f943f2f163ba..5cfd5214b982 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java @@ -45,7 +45,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { if (dimValues instanceof List) { throw new UnsupportedOperationException("Numeric columns do not support multivalue rows."); @@ -54,7 +54,7 @@ public Double processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimVa if (d == null) { hasNulls = NullHandling.sqlCompatible(); } - return d; + return new EncodedDimensionValue<>(d, Double.BYTES); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/EncodedDimensionValue.java b/processing/src/main/java/org/apache/druid/segment/EncodedDimensionValue.java new file mode 100644 index 000000000000..c0dcac9bee20 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/EncodedDimensionValue.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import javax.annotation.Nullable; + +/** + * @param Encoded key component type + */ +public class EncodedDimensionValue +{ + @Nullable + private final K value; + private final long incrementalSizeBytes; + + EncodedDimensionValue(@Nullable K value, long incrementalSizeBytes) + { + this.value = value; + this.incrementalSizeBytes = incrementalSizeBytes; + } + + public K getValue() + { + return value; + } + + public long getIncrementalSizeBytes() + { + return incrementalSizeBytes; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java index 21bd1a65c1b7..3989156e0d97 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java @@ -45,7 +45,7 @@ public class FloatDimensionIndexer implements DimensionIndexer processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { if (dimValues instanceof List) { throw new UnsupportedOperationException("Numeric columns do not support multivalue rows."); @@ -55,7 +55,7 @@ public Float processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimVal if (f == null) { hasNulls = NullHandling.sqlCompatible(); } - return f; + return new EncodedDimensionValue<>(f, Float.BYTES); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java index bce27c27fcc9..7456725f4275 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java @@ -46,7 +46,7 @@ public class LongDimensionIndexer implements DimensionIndexer @Nullable @Override - public Long processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) + public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { if (dimValues instanceof List) { throw new UnsupportedOperationException("Numeric columns do not support multivalue rows."); @@ -56,7 +56,7 @@ public Long processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValu if (l == null) { hasNulls = NullHandling.sqlCompatible(); } - return l; + return new EncodedDimensionValue<>(l, Long.BYTES); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 108db6e72ff8..4e09105018e6 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -70,10 +70,11 @@ public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean has } @Override - public int[] processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) + public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { final int[] encodedDimensionValues; final int oldDictSize = dimLookup.size(); + final long oldDictSizeInBytes = dimLookup.sizeInBytes(); if (dimValues == null) { final int nullId = dimLookup.getId(null); @@ -122,7 +123,10 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimVal sortedLookup = null; } - return encodedDimensionValues; + // incremental size = increase in size of dimension dict + size of encoded array + long keyComponentSizeBytes = (dimLookup.sizeInBytes() - oldDictSizeInBytes) + + (long) encodedDimensionValues.length * Integer.BYTES; + return new EncodedDimensionValue<>(encodedDimensionValues, keyComponentSizeBytes); } @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 index faf164f45ab2..5260669f8bb6 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 @@ -34,6 +34,7 @@ public abstract class AppendableIndexBuilder protected boolean sortFacts = true; protected int maxRowCount = 0; protected long maxBytesInMemory = 0; + protected boolean useMaxMemoryEstimates = true; protected final Logger log = new Logger(this.getClass()); @@ -106,6 +107,12 @@ public AppendableIndexBuilder setMaxBytesInMemory(final long maxBytesInMemory) return this; } + public AppendableIndexBuilder setUseMaxMemoryEstimates(final boolean useMaxMemoryEstimates) + { + this.useMaxMemoryEstimates = useMaxMemoryEstimates; + return this; + } + public void validate() { if (maxRowCount <= 0) { 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 a94ae1c23493..7715c4652fe3 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 @@ -57,6 +57,7 @@ import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.DoubleColumnSelector; +import org.apache.druid.segment.EncodedDimensionValue; import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.LongColumnSelector; @@ -478,6 +479,11 @@ public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededExcep return add(row, false); } + public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException + { + return add(row, false, true); + } + /** * Adds a new row. The row might correspond with another row that already exists, in which case this will * update that row instead of inserting a new one. @@ -486,14 +492,19 @@ public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededExcep * Calls to add() are thread safe. *

* - * @param row the row of data to add - * @param skipMaxRowsInMemoryCheck whether or not skip the check of rows exceeding the max rows limit + * @param row the row of data to add + * @param skipMaxRowsInMemoryCheck whether or not to skip the check of rows exceeding the max rows limit + * @param useMaxMemoryEstimates whether or not to use maximum values to estimate memory * @return the number of rows in the data set after adding the InputRow. If any parse failure occurs, a {@link ParseException} is returned in {@link IncrementalIndexAddResult}. * @throws IndexSizeExceededException this exception is thrown once it reaches max rows limit and skipMaxRowsInMemoryCheck is set to false. */ - public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException + public IncrementalIndexAddResult add( + InputRow row, + boolean skipMaxRowsInMemoryCheck, + boolean useMaxMemoryEstimates + ) throws IndexSizeExceededException { - IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row); + IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row, useMaxMemoryEstimates); final AddToFactsResult addToFactsResult = addToFacts( row, incrementalIndexRowResult.getIncrementalIndexRow(), @@ -516,6 +527,11 @@ public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCh @VisibleForTesting IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) + { + return toIncrementalIndexRow(row, true); + } + + private IncrementalIndexRowResult toIncrementalIndexRow(InputRow row, boolean useMaxMemoryEstimates) { row = formatRow(row); if (row.getTimestampFromEpoch() < minTimestamp) { @@ -557,12 +573,17 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) DimensionIndexer indexer = desc.getIndexer(); Object dimsKey = null; try { - dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent(row.getRaw(dimension), true); + final EncodedDimensionValue encodedDimensionValue + = indexer.processRowValsToUnsortedEncodedKeyComponent(row.getRaw(dimension), true); + dimsKey = encodedDimensionValue.getValue(); + final long keySizeDelta = useMaxMemoryEstimates + ? indexer.estimateEncodedKeyComponentSize(dimsKey) + : encodedDimensionValue.getIncrementalSizeBytes(); + dimsKeySize += keySizeDelta; } catch (ParseException pe) { parseExceptionMessages.add(pe.getMessage()); } - dimsKeySize += indexer.estimateEncodedKeyComponentSize(dimsKey); if (wasNewDim) { // unless this is the first row we are processing, all newly discovered columns will be sparse if (maxIngestedEventTime != null) { diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index 987ee5f8bf7c..4c4713cae9f1 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java @@ -113,8 +113,7 @@ void setRowIndex(int rowIndex) } /** - * bytesInMemory estimates the size of IncrementalIndexRow key, it takes into account the timestamp(long), - * dims(Object Array) and dimensionDescsList(List). Each of these are calculated as follows: + * Estimates the size of IncrementalIndexRow key by adding the following components: *

    *
  • timestamp : Long.BYTES *
  • dims array : Integer.BYTES * array length + Long.BYTES (dims object) + dimsKeySize(passed via constructor) @@ -126,7 +125,7 @@ void setRowIndex(int rowIndex) */ public long estimateBytesInMemory() { - long sizeInBytes = Long.BYTES + ((long) Integer.BYTES) * dims.length + Long.BYTES + Long.BYTES; + long sizeInBytes = Long.BYTES + Integer.BYTES * (long) dims.length + Long.BYTES + Long.BYTES; sizeInBytes += dimsKeySize; return sizeInBytes; } @@ -137,17 +136,10 @@ public String toString() return "IncrementalIndexRow{" + "timestamp=" + DateTimes.utc(timestamp) + ", dims=" + Lists.transform( - Arrays.asList(dims), new Function() - { - @Override - public Object apply(@Nullable Object input) - { - if (input == null || Array.getLength(input) == 0) { - return Collections.singletonList("null"); - } - return Collections.singletonList(input); - } - } + Arrays.asList(dims), + input -> (input == null || Array.getLength(input) == 0) + ? Collections.singletonList("null") + : Collections.singletonList(input) ) + '}'; } 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 bb24ae4cadde..6e42d74db10b 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.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; @@ -77,6 +78,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex private final long maxBytesPerRowForAggregators; protected final int maxRowCount; protected final long maxBytesInMemory; + private final boolean useMaxMemoryEstimates; @Nullable private volatile Map selectors; @@ -89,7 +91,8 @@ public class OnheapIncrementalIndex extends IncrementalIndex boolean concurrentEventAdd, boolean sortFacts, int maxRowCount, - long maxBytesInMemory + long maxBytesInMemory, + boolean useMaxMemoryEstimates ) { super(incrementalIndexSchema, deserializeComplexMetrics, concurrentEventAdd); @@ -97,7 +100,9 @@ public class OnheapIncrementalIndex extends IncrementalIndex this.maxBytesInMemory = maxBytesInMemory == 0 ? Long.MAX_VALUE : maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) : new PlainFactsHolder(sortFacts, dimsComparator()); - maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); + maxBytesPerRowForAggregators = + useMaxMemoryEstimates ? getMaxBytesPerRowForAggregators(incrementalIndexSchema) : 0; + this.useMaxMemoryEstimates = useMaxMemoryEstimates; } /** @@ -175,20 +180,21 @@ protected AddToFactsResult addToFacts( Aggregator[] aggs; final AggregatorFactory[] metrics = getMetrics(); final AtomicInteger numEntries = getNumEntries(); - final AtomicLong sizeInBytes = getBytesInMemory(); + final AtomicLong totalSizeInBytes = getBytesInMemory(); if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { aggs = concurrentGet(priorIndex); - doAggregate(metrics, aggs, rowContainer, row, parseExceptionMessages); + long aggSizeDelta = doAggregate(metrics, aggs, rowContainer, row, parseExceptionMessages); + totalSizeInBytes.addAndGet(useMaxMemoryEstimates ? 0 : aggSizeDelta); } else { aggs = new Aggregator[metrics.length]; - factorizeAggs(metrics, aggs, rowContainer, row); - doAggregate(metrics, aggs, rowContainer, row, parseExceptionMessages); + long aggSizeForRow = factorizeAggs(metrics, aggs, rowContainer, row); + aggSizeForRow += doAggregate(metrics, aggs, rowContainer, row, parseExceptionMessages); final int rowIndex = indexIncrement.getAndIncrement(); concurrentSet(rowIndex, aggs); // Last ditch sanity checks - if ((numEntries.get() >= maxRowCount || sizeInBytes.get() >= maxBytesInMemory) + if ((numEntries.get() >= maxRowCount || totalSizeInBytes.get() >= maxBytesInMemory) && facts.getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX && !skipMaxRowsInMemoryCheck) { throw new IndexSizeExceededException( @@ -200,38 +206,28 @@ protected AddToFactsResult addToFacts( final int prev = facts.putIfAbsent(key, rowIndex); if (IncrementalIndexRow.EMPTY_ROW_INDEX == prev) { numEntries.incrementAndGet(); - long estimatedRowSize = estimateRowSizeInBytes(key, maxBytesPerRowForAggregators); - sizeInBytes.addAndGet(estimatedRowSize); } else { - // We lost a race + // This would happen in a race condition where there are multiple write threads + // which could be possible in case of GroupBy v1 strategy parseExceptionMessages.clear(); aggs = concurrentGet(prev); - doAggregate(metrics, aggs, rowContainer, row, parseExceptionMessages); + aggSizeForRow = doAggregate(metrics, aggs, rowContainer, row, parseExceptionMessages); + // Free up the misfire concurrentRemove(rowIndex); // This is expected to occur ~80% of the time in the worst scenarios } - } - return new AddToFactsResult(numEntries.get(), sizeInBytes.get(), parseExceptionMessages); - } + // For a new key, row size = key size + aggregator size + overhead + final long estimatedSizeOfAggregators = + useMaxMemoryEstimates ? maxBytesPerRowForAggregators : aggSizeForRow; + final long rowSize = key.estimateBytesInMemory() + + estimatedSizeOfAggregators + + ROUGH_OVERHEAD_PER_MAP_ENTRY; + totalSizeInBytes.addAndGet(rowSize); + } - /** - * Gives an estimated size of row in bytes, it accounts for: - *
      - *
    • overhead per Map Entry - *
    • TimeAndDims key size - *
    • aggregator size - *
    - * - * @param key TimeAndDims key - * @param maxBytesPerRowForAggregators max size per aggregator - * - * @return estimated size of row - */ - private long estimateRowSizeInBytes(IncrementalIndexRow key, long maxBytesPerRowForAggregators) - { - return ROUGH_OVERHEAD_PER_MAP_ENTRY + key.estimateBytesInMemory() + maxBytesPerRowForAggregators; + return new AddToFactsResult(numEntries.get(), totalSizeInBytes.get(), parseExceptionMessages); } @Override @@ -240,22 +236,46 @@ public int getLastRowIndex() return indexIncrement.get() - 1; } - private void factorizeAggs( + /** + * Creates aggregators for the given aggregator factories. + * + * @return Total initial size in bytes required by all the aggregators. + * This value is non-zero only when {@link #useMaxMemoryEstimates} is false. + */ + private long factorizeAggs( AggregatorFactory[] metrics, Aggregator[] aggs, ThreadLocal rowContainer, InputRow row ) { + long totalInitialSizeBytes = 0L; rowContainer.set(row); for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; - aggs[i] = agg.factorize(selectors.get(agg.getName())); + + if (useMaxMemoryEstimates) { + aggs[i] = agg.factorize(selectors.get(agg.getName())); + } else { + AggregatorAndSize aggregatorAndSize = + agg.factorizeWithSize(selectors.get(agg.getName())); + aggs[i] = aggregatorAndSize.getAggregator(); + totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes(); + } } rowContainer.set(null); + + return totalInitialSizeBytes; } - private void doAggregate( + /** + * Performs aggregation for all of the aggregators. + * + * @return Total incremental memory in bytes required by this step of the + * aggregation. The returned value is non-zero only if + * {@link #useMaxMemoryEstimates} is false. + */ + private long doAggregate( AggregatorFactory[] metrics, Aggregator[] aggs, ThreadLocal rowContainer, @@ -265,11 +285,16 @@ private void doAggregate( { rowContainer.set(row); + long totalIncrementalBytes = 0L; for (int i = 0; i < aggs.length; i++) { final Aggregator agg = aggs[i]; synchronized (agg) { try { - agg.aggregate(); + if (useMaxMemoryEstimates) { + agg.aggregate(); + } else { + totalIncrementalBytes += agg.aggregateWithSize(); + } } catch (ParseException e) { // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. @@ -280,6 +305,7 @@ private void doAggregate( } rowContainer.set(null); + return totalIncrementalBytes; } private void closeAggregators() @@ -513,7 +539,8 @@ protected OnheapIncrementalIndex buildInner() concurrentEventAdd, sortFacts, maxRowCount, - maxBytesInMemory + maxBytesInMemory, + useMaxMemoryEstimates ); } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 7b7783917d37..13c6a6834916 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -123,7 +123,8 @@ public MapIncrementalIndex( concurrentEventAdd, sortFacts, maxRowCount, - maxBytesInMemory + maxBytesInMemory, + false ); } @@ -145,7 +146,8 @@ public MapIncrementalIndex( false, true, maxRowCount, - maxBytesInMemory + maxBytesInMemory, + false ); } 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 f0ee59f59e00..a6c9f7e507cb 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 @@ -174,6 +174,7 @@ public class AppenderatorImpl implements Appenderator private volatile Throwable persistError; private final boolean isOpenSegments; + private final boolean useMaxMemoryEstimates; /** * Use next Map to store metadata (File, SegmentId) for a hydrant for batch appenderator * in order to facilitate the mapping of the QueryableIndex associated with a given hydrant @@ -209,7 +210,8 @@ public class AppenderatorImpl implements Appenderator Cache cache, RowIngestionMeters rowIngestionMeters, ParseExceptionHandler parseExceptionHandler, - boolean isOpenSegments + boolean isOpenSegments, + boolean useMaxMemoryEstimates ) { this.myId = id; @@ -226,6 +228,7 @@ public class AppenderatorImpl implements Appenderator this.rowIngestionMeters = Preconditions.checkNotNull(rowIngestionMeters, "rowIngestionMeters"); this.parseExceptionHandler = Preconditions.checkNotNull(parseExceptionHandler, "parseExceptionHandler"); this.isOpenSegments = isOpenSegments; + this.useMaxMemoryEstimates = useMaxMemoryEstimates; if (sinkQuerySegmentWalker == null) { this.sinkTimeline = new VersionedIntervalTimeline<>( @@ -497,6 +500,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, + useMaxMemoryEstimates, null ); bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal)); @@ -1301,6 +1305,7 @@ private Object bootstrapSinksFromDisk() tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, + useMaxMemoryEstimates, null, hydrants ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 08949cceaa5c..9b0064fbcfa5 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -59,7 +59,8 @@ public static Appenderator createRealtime( CacheConfig cacheConfig, CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { return new StreamAppenderator( @@ -88,7 +89,8 @@ public static Appenderator createRealtime( indexMerger, cache, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } @@ -102,7 +104,8 @@ public static Appenderator createOffline( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { // Use newest, slated to be the permanent batch appenderator but for now keeping it as a non-default @@ -118,7 +121,8 @@ public static Appenderator createOffline( indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } @@ -132,7 +136,8 @@ public static Appenderator createOpenSegmentsOffline( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { // fallback to original code known to be working, this is just a fallback option in case new @@ -152,7 +157,8 @@ public static Appenderator createOpenSegmentsOffline( null, rowIngestionMeters, parseExceptionHandler, - true + true, + useMaxMemoryEstimates ); } @@ -166,7 +172,8 @@ public static Appenderator createClosedSegmentsOffline( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { return new AppenderatorImpl( @@ -183,7 +190,8 @@ public static Appenderator createClosedSegmentsOffline( null, rowIngestionMeters, parseExceptionHandler, - false + false, + useMaxMemoryEstimates ); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java index f578a20e3604..75e078e1bb94 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorsManager.java @@ -81,7 +81,8 @@ Appenderator createRealtimeAppenderatorForTask( CacheConfig cacheConfig, CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ); /** @@ -97,7 +98,8 @@ Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ); Appenderator createClosedSegmentsOfflineAppenderatorForTask( @@ -110,7 +112,8 @@ Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ); Appenderator createOfflineAppenderatorForTask( @@ -123,7 +126,8 @@ Appenderator createOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ); /** diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java index ae5b06fcb399..5d19550e35b0 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderator.java @@ -115,6 +115,7 @@ public class BatchAppenderator implements Appenderator private final IndexMerger indexMerger; private final long maxBytesTuningConfig; private final boolean skipBytesInMemoryOverheadCheck; + private final boolean useMaxMemoryEstimates; private volatile ListeningExecutorService persistExecutor = null; private volatile ListeningExecutorService pushExecutor = null; @@ -160,7 +161,8 @@ public class BatchAppenderator implements Appenderator IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { this.myId = id; @@ -177,6 +179,7 @@ public class BatchAppenderator implements Appenderator maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); maxPendingPersists = tuningConfig.getMaxPendingPersists(); + this.useMaxMemoryEstimates = useMaxMemoryEstimates; } @Override @@ -465,6 +468,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, + useMaxMemoryEstimates, null ); bytesCurrentlyInMemory += calculateSinkMemoryInUsed(); @@ -1032,6 +1036,7 @@ private Sink getSinkForIdentifierPath(SegmentIdWithShardSpec identifier, File id tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, + useMaxMemoryEstimates, null, hydrants ); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java index eb308528909e..b6b5a031cb5e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java @@ -73,7 +73,8 @@ public Appenderator build(DataSchema schema, RealtimeTuningConfig config, FireDe false, config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE, 0 - ) + ), + true ); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java index d99aa81672d8..7698d41c8a04 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java @@ -124,7 +124,8 @@ public Appenderator build( false, config.isReportParseExceptions() ? 0 : Integer.MAX_VALUE, 0 - ) + ), + true ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java index 699c32478a9e..10939cf5356c 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DummyForInjectionAppenderatorsManager.java @@ -72,7 +72,8 @@ public Appenderator createRealtimeAppenderatorForTask( CacheConfig cacheConfig, CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { throw new UOE(ERROR_MSG); @@ -89,7 +90,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { throw new UOE(ERROR_MSG); @@ -106,7 +108,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { throw new UOE(ERROR_MSG); @@ -123,7 +126,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { throw new UOE(ERROR_MSG); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java index b666565125d1..82df08c665a9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/PeonAppenderatorsManager.java @@ -78,7 +78,8 @@ public Appenderator createRealtimeAppenderatorForTask( CacheConfig cacheConfig, CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { if (realtimeAppenderator != null) { @@ -104,7 +105,8 @@ public Appenderator createRealtimeAppenderatorForTask( cacheConfig, cachePopulatorStats, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); } return realtimeAppenderator; @@ -121,7 +123,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -138,7 +141,8 @@ public Appenderator createOfflineAppenderatorForTask( indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); return batchAppenderator; } @@ -155,7 +159,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -172,7 +177,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); return batchAppenderator; } @@ -189,7 +195,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { // CompactionTask does run multiple sub-IndexTasks, so we allow multiple batch appenderators @@ -206,7 +213,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( indexIO, indexMerger, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); return batchAppenderator; } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java index 26b492ae149f..70a60a77d85e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderator.java @@ -137,6 +137,7 @@ public class StreamAppenderator implements Appenderator private final VersionedIntervalTimeline sinkTimeline; private final long maxBytesTuningConfig; private final boolean skipBytesInMemoryOverheadCheck; + private final boolean useMaxMemoryEstimates; private final QuerySegmentWalker texasRanger; // This variable updated in add(), persist(), and drop() @@ -185,7 +186,8 @@ public class StreamAppenderator implements Appenderator IndexMerger indexMerger, Cache cache, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { this.myId = id; @@ -212,6 +214,7 @@ public class StreamAppenderator implements Appenderator maxBytesTuningConfig = tuningConfig.getMaxBytesInMemoryOrDefault(); skipBytesInMemoryOverheadCheck = tuningConfig.isSkipBytesInMemoryOverheadCheck(); + this.useMaxMemoryEstimates = useMaxMemoryEstimates; } @Override @@ -464,6 +467,7 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, + useMaxMemoryEstimates, null ); bytesCurrentlyInMemory.addAndGet(calculateSinkMemoryInUsed(retVal)); @@ -1223,6 +1227,7 @@ private Object bootstrapSinksFromDisk() tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, + useMaxMemoryEstimates, null, hydrants ); 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 5a7a61417607..3cc95324adca 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 @@ -165,7 +165,8 @@ public Appenderator createRealtimeAppenderatorForTask( CacheConfig cacheConfig, CachePopulatorStats cachePopulatorStats, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { synchronized (this) { @@ -187,7 +188,8 @@ public Appenderator createRealtimeAppenderatorForTask( wrapIndexMerger(indexMerger), cache, rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); datasourceBundle.addAppenderator(taskId, appenderator); @@ -206,7 +208,8 @@ public Appenderator createOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { synchronized (this) { @@ -225,7 +228,8 @@ public Appenderator createOfflineAppenderatorForTask( indexIO, wrapIndexMerger(indexMerger), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; @@ -243,7 +247,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { synchronized (this) { @@ -262,7 +267,8 @@ public Appenderator createOpenSegmentsOfflineAppenderatorForTask( indexIO, wrapIndexMerger(indexMerger), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; @@ -280,7 +286,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( IndexIO indexIO, IndexMerger indexMerger, RowIngestionMeters rowIngestionMeters, - ParseExceptionHandler parseExceptionHandler + ParseExceptionHandler parseExceptionHandler, + boolean useMaxMemoryEstimates ) { synchronized (this) { @@ -299,7 +306,8 @@ public Appenderator createClosedSegmentsOfflineAppenderatorForTask( indexIO, wrapIndexMerger(indexMerger), rowIngestionMeters, - parseExceptionHandler + parseExceptionHandler, + useMaxMemoryEstimates ); datasourceBundle.addAppenderator(taskId, appenderator); return appenderator; 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 757266c3a5ee..b5c32f481932 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 @@ -271,6 +271,7 @@ private Sink getSink(long timestamp) config.getAppendableIndexSpec(), config.getMaxRowsInMemory(), config.getMaxBytesInMemoryOrDefault(), + true, config.getDedupColumn() ); addSink(retVal); @@ -739,6 +740,7 @@ public int compare(File o1, File o2) config.getAppendableIndexSpec(), config.getMaxRowsInMemory(), config.getMaxBytesInMemoryOrDefault(), + true, 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 ccf5e4c5421b..656347920de7 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 @@ -67,6 +67,7 @@ public class Sink implements Iterable, Overshadowable private final AppendableIndexSpec appendableIndexSpec; private final int maxRowsInMemory; private final long maxBytesInMemory; + private final boolean useMaxMemoryEstimates; private final CopyOnWriteArrayList hydrants = new CopyOnWriteArrayList<>(); private final LinkedHashSet dimOrder = new LinkedHashSet<>(); private final AtomicInteger numRowsExcludingCurrIndex = new AtomicInteger(); @@ -84,6 +85,7 @@ public Sink( AppendableIndexSpec appendableIndexSpec, int maxRowsInMemory, long maxBytesInMemory, + boolean useMaxMemoryEstimates, String dedupColumn ) { @@ -95,6 +97,7 @@ public Sink( appendableIndexSpec, maxRowsInMemory, maxBytesInMemory, + useMaxMemoryEstimates, dedupColumn, Collections.emptyList() ); @@ -108,6 +111,7 @@ public Sink( AppendableIndexSpec appendableIndexSpec, int maxRowsInMemory, long maxBytesInMemory, + boolean useMaxMemoryEstimates, String dedupColumn, List hydrants ) @@ -119,6 +123,7 @@ public Sink( this.appendableIndexSpec = appendableIndexSpec; this.maxRowsInMemory = maxRowsInMemory; this.maxBytesInMemory = maxBytesInMemory; + this.useMaxMemoryEstimates = useMaxMemoryEstimates; this.dedupColumn = dedupColumn; int maxCount = -1; @@ -176,7 +181,7 @@ public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCh return Plumber.DUPLICATE; } - return index.add(row, skipMaxRowsInMemoryCheck); + return index.add(row, skipMaxRowsInMemoryCheck, useMaxMemoryEstimates); } } @@ -334,6 +339,7 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) .setIndexSchema(indexSchema) .setMaxRowCount(maxRowsInMemory) .setMaxBytesInMemory(maxBytesInMemory) + .setUseMaxMemoryEstimates(useMaxMemoryEstimates) .build(); final FireHydrant old; diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java index a3c09c2e27df..c2bddbac3b42 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/ClosedSegmensSinksBatchAppenderatorTester.java @@ -246,7 +246,8 @@ public Map makeLoadSpec(URI uri) indexIO, indexMerger, rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + true ); } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java index ed89ac76fe1f..754c88ed5e88 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/OpenAndClosedSegmentsAppenderatorTester.java @@ -222,7 +222,8 @@ public Map makeLoadSpec(URI uri) indexIO, indexMerger, rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)); + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + true); } else { appenderator = Appenderators.createClosedSegmentsOffline( schema.getDataSource(), @@ -234,7 +235,8 @@ public Map makeLoadSpec(URI uri) indexIO, indexMerger, rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0)); + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + true); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java index 003e2fb2269a..859c57b2f325 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorTester.java @@ -290,7 +290,8 @@ ScanQuery.class, new ScanQueryRunnerFactory( new CacheConfig(), new CachePopulatorStats(), rowIngestionMeters, - new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0) + new ParseExceptionHandler(rowIngestionMeters, false, Integer.MAX_VALUE, 0), + true ); } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java index 8f95eacb7ce0..5828d0fba785 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/UnifiedIndexerAppenderatorsManagerTest.java @@ -111,7 +111,8 @@ public void setup() TestHelper.getTestIndexIO(), TestHelper.getTestIndexMergerV9(OnHeapMemorySegmentWriteOutMediumFactory.instance()), new NoopRowIngestionMeters(), - new ParseExceptionHandler(new NoopRowIngestionMeters(), false, 0, 0) + new ParseExceptionHandler(new NoopRowIngestionMeters(), false, 0, 0), + true ); } 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 80f546103491..c2b18638f464 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 @@ -283,6 +283,7 @@ private void testPersist(final Object commitMetadata) throws Exception tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), tuningConfig.getMaxBytesInMemoryOrDefault(), + true, tuningConfig.getDedupColumn() ); plumber.getSinks().put(0L, sink); @@ -329,6 +330,7 @@ public void testPersistFails() throws Exception tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), tuningConfig.getMaxBytesInMemoryOrDefault(), + true, tuningConfig.getDedupColumn() ); plumber.getSinks().put(0L, sink); @@ -380,6 +382,7 @@ private void testPersistHydrantGapsHelper(final Object commitMetadata) throws Ex tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), tuningConfig.getMaxBytesInMemoryOrDefault(), + true, 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 de12c4c60b73..2d3bdc4ae7ee 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 @@ -94,6 +94,7 @@ public void testSwap() throws Exception tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), tuningConfig.getMaxBytesInMemoryOrDefault(), + true, tuningConfig.getDedupColumn() ); @@ -250,6 +251,7 @@ public void testDedup() throws Exception tuningConfig.getAppendableIndexSpec(), tuningConfig.getMaxRowsInMemory(), tuningConfig.getMaxBytesInMemoryOrDefault(), + true, tuningConfig.getDedupColumn() ); From 78a2f0eeae2ac62d0d0936f9e1cb7bd15bab50b3 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 12 Jan 2022 00:24:44 +0530 Subject: [PATCH 02/15] Fix LGTM errors --- .../aggregation/datasketches/theta/SketchAggregator.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java index 6902fc2f8826..b42093b8fe6c 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -53,7 +53,7 @@ public class SketchAggregator implements Aggregator sketchField.setAccessible(true); } catch (NoSuchFieldException | ClassNotFoundException e) { - throw new ISE("Could not initialize SketchAggregator", e); + throw new ISE(e, "Could not initialize SketchAggregator"); } } @@ -74,7 +74,7 @@ private void initSketch() sketch = (Sketch) sketchField.get(union); } catch (IllegalAccessException e) { - throw new ISE("Could not initialize sketch field in SketchAggregator", e); + throw new ISE(e, "Could not initialize sketch field in SketchAggregator"); } } @@ -199,7 +199,7 @@ static void updateUnion(Union union, Object update) public long getInitialSizeBytes() { // SketchAggregator has 3 references and an int - return 3 * Long.BYTES + Integer.BYTES; + return 3L * Long.BYTES + Integer.BYTES; } } From 598cfb50a21ace169415e48503476f7405ab170a Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 12 Jan 2022 11:08:22 +0530 Subject: [PATCH 03/15] Fix SketchAggregatorFactoryTest --- .../datasketches/theta/SketchAggregatorFactoryTest.java | 4 ++-- .../org/apache/druid/query/aggregation/AggregatorFactory.java | 3 ++- .../apache/druid/segment/incremental/IncrementalIndexRow.java | 2 -- .../druid/segment/incremental/OnheapIncrementalIndex.java | 2 +- 4 files changed, 5 insertions(+), 6 deletions(-) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java index b5d3a1a2dd2d..30cbffa642d2 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java @@ -74,10 +74,10 @@ public void testFactorizeSized() EasyMock.replay(colSelectorFactory); AggregatorAndSize aggregatorAndSize = AGGREGATOR_16384.factorizeWithSize(colSelectorFactory); - Assert.assertEquals(591, aggregatorAndSize.getInitialSizeBytes()); + Assert.assertEquals(28, aggregatorAndSize.getInitialSizeBytes()); aggregatorAndSize = AGGREGATOR_32768.factorizeWithSize(colSelectorFactory); - Assert.assertEquals(1103, aggregatorAndSize.getInitialSizeBytes()); + Assert.assertEquals(28, aggregatorAndSize.getInitialSizeBytes()); } @Test diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index a65f1e02d897..9a015ba33371 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -74,7 +74,8 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFact * * @return AggregatorAndSize which contains the actual aggregator and its initial size. */ - public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) { + public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) + { return new AggregatorAndSize(factorize(metricFactory), getMaxIntermediateSize()); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index 4c4713cae9f1..8ca1feb4365f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java @@ -19,12 +19,10 @@ package org.apache.druid.segment.incremental; -import com.google.common.base.Function; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.DimensionIndexer; -import javax.annotation.Nullable; import java.lang.reflect.Array; import java.util.Arrays; import java.util.Collections; 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 6e42d74db10b..d1b3fa65e810 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 @@ -30,9 +30,9 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; From 9908c0baa918b03b61f570329e0c412a404448db Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 17 Jan 2022 11:03:18 +0530 Subject: [PATCH 04/15] Fix estimations, add javadoc --- .../datasketches/theta/SketchAggregator.java | 22 ++++--- .../theta/SketchAggregatorFactory.java | 2 +- .../theta/SketchAggregatorFactoryTest.java | 2 +- .../druid/indexing/common/task/Tasks.java | 9 +++ .../overlord/sampler/InputSourceSampler.java | 2 +- .../druid/query/aggregation/Aggregator.java | 11 ++++ .../query/aggregation/AggregatorAndSize.java | 11 +++- .../query/aggregation/AggregatorFactory.java | 8 +++ .../DictionaryEncodedColumnIndexer.java | 20 +++++- .../druid/segment/DimensionDictionary.java | 22 +++---- .../druid/segment/DimensionIndexer.java | 39 ++++++----- .../druid/segment/DoubleDimensionIndexer.java | 4 +- .../druid/segment/EncodedKeyComponent.java | 64 +++++++++++++++++++ .../druid/segment/FloatDimensionIndexer.java | 4 +- .../druid/segment/LongDimensionIndexer.java | 4 +- ...ue.java => StringDimensionDictionary.java} | 28 ++------ .../druid/segment/StringDimensionIndexer.java | 5 +- .../segment/incremental/IncrementalIndex.java | 11 +--- .../incremental/OnheapIncrementalIndex.java | 31 +++++++++ .../OnheapIncrementalIndexBenchmark.java | 4 +- 20 files changed, 220 insertions(+), 83 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java rename processing/src/main/java/org/apache/druid/segment/{EncodedDimensionValue.java => StringDimensionDictionary.java} (64%) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java index b42093b8fe6c..de582992b1e7 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -45,12 +45,13 @@ public class SketchAggregator implements Aggregator private Sketch sketch; @Nullable - private static final Field sketchField; + private static final Field SKETCH_FIELD; + static { try { - sketchField = Class.forName("org.apache.datasketches.theta.UnionImpl") - .getDeclaredField("gadget_"); - sketchField.setAccessible(true); + SKETCH_FIELD = Class.forName("org.apache.datasketches.theta.UnionImpl") + .getDeclaredField("gadget_"); + SKETCH_FIELD.setAccessible(true); } catch (NoSuchFieldException | ClassNotFoundException e) { throw new ISE(e, "Could not initialize SketchAggregator"); @@ -71,7 +72,7 @@ private void initUnion() private void initSketch() { try { - sketch = (Sketch) sketchField.get(union); + sketch = (Sketch) SKETCH_FIELD.get(union); } catch (IllegalAccessException e) { throw new ISE(e, "Could not initialize sketch field in SketchAggregator"); @@ -105,8 +106,9 @@ public long aggregateWithSize() if (union == null) { initUnion(); - // Fields in UnionImpl: a sketch reference, a short, a long and a boolean - unionSizeDelta = Long.BYTES + Short.BYTES + Long.BYTES + 1; + // Size of UnionImpl = 16B (object header) + 8B (sketch ref) + 2B (short) + // + 8B (long) + 1B (boolean) + 5B (padding) = 40B + unionSizeDelta = 40L; } long initialSketchSize = 0; @@ -198,8 +200,10 @@ static void updateUnion(Union union, Object update) */ public long getInitialSizeBytes() { - // SketchAggregator has 3 references and an int - return 3L * Long.BYTES + Integer.BYTES; + // Size = 16B (object header) + 24B (3 refs) + 4B (int size) = 44B + // Due to 8-byte alignment, size = 48B + // (see https://www.baeldung.com/java-memory-layout) + return 48L; } } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java index 64a7fd5014a9..2b0dff497637 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java @@ -29,10 +29,10 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregateCombiner; import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.ObjectAggregateCombiner; -import org.apache.druid.query.aggregation.AggregatorAndSize; import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnInspector; diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java index 30cbffa642d2..c8fb91ca7a97 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java @@ -22,8 +22,8 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.query.Druids; -import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.AggregatorAndSize; +import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.theta.oldapi.OldSketchBuildAggregatorFactory; import org.apache.druid.query.aggregation.datasketches.theta.oldapi.OldSketchMergeAggregatorFactory; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java index 55e46105d975..99c7a4db437d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java @@ -52,6 +52,15 @@ public class Tasks public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout"; public static final String FORCE_TIME_CHUNK_LOCK_KEY = "forceTimeChunkLock"; public static final String USE_SHARED_LOCK = "useSharedLock"; + + /** + * Context flag denoting if maximum possible values should be used to estimate + * on-heap memory usage while indexing. Refer to OnHeapIncrementalIndex for + * more details. + * + * The value of this flag is true by default which corresponds to the old method + * of estimation. + */ public static final String USE_MAX_MEMORY_ESTIMATES = "useMaxMemoryEstimates"; /** 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 21685c4d7455..c864658a2610 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 @@ -154,7 +154,7 @@ public SamplerResponse sample( //keep the index of the row to be added to responseRows for further use final int rowIndex = responseRows.size(); - IncrementalIndexAddResult addResult = index.add(new SamplerInputRow(row, rowIndex), true); + IncrementalIndexAddResult addResult = index.add(new SamplerInputRow(row, rowIndex), true, true); if (addResult.hasParseException()) { responseRows.add(new SamplerResponseRow( rawColumns, diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java index c4b225a02569..e59a93cbf34d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java @@ -40,8 +40,19 @@ @ExtensionPoint public interface Aggregator extends Closeable { + /** + * Performs aggregation. + */ void aggregate(); + /** + * Performs aggregation and returns the increase in required on-heap memory + * caused by this aggregation step. + *

    + * The default implementation of this method calls {@link #aggregate()} and returns 0. + * + * @return Increase in required on-heap memory caused by this aggregation step. + */ default long aggregateWithSize() { aggregate(); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java index 6259c7a75a4f..22b8af1f527a 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorAndSize.java @@ -25,12 +25,14 @@ */ public class AggregatorAndSize { - - // TODO: include default overhead for object sizes - private final Aggregator aggregator; private final long initialSizeBytes; + /** + * @param aggregator Aggregator + * @param initialSizeBytes Initial size in bytes (including JVM object overheads) + * required by the aggregator. + */ public AggregatorAndSize(Aggregator aggregator, long initialSizeBytes) { this.aggregator = aggregator; @@ -42,6 +44,9 @@ public Aggregator getAggregator() return aggregator; } + /** + * Initial size of the aggregator in bytes including JVM object overheads. + */ public long getInitialSizeBytes() { return initialSizeBytes; diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 9a015ba33371..59c850ba810c 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -71,6 +71,14 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFact /** * Creates an {@link Aggregator} based on the provided column selector factory. + * The returned value is a holder object which contains both the aggregator + * and its initial size in bytes. The callers can then invoke + * {@link Aggregator#aggregateWithSize()} to perform aggregation and get back + * the incremental memory required in each aggregate call. Combined with the + * initial size, this gives the total on-heap memory required by the aggregator. + * + * This flow does not require invoking {@link #guessAggregatorHeapFootprint(long)} + * which tends to over-estimate the required memory. * * @return AggregatorAndSize which contains the actual aggregator and its initial size. */ diff --git a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java index d0a5aed86161..f78ea9599897 100644 --- a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java @@ -19,6 +19,7 @@ package org.apache.druid.segment; +import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -29,6 +30,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; import javax.annotation.Nullable; +import javax.validation.constraints.NotNull; import java.util.Iterator; /** @@ -43,9 +45,25 @@ public abstract class DictionaryEncodedColumnIndexer sortedLookup; + /** + * Creates a new DictionaryEncodedColumnIndexer with the default implementation + * of {@link DimensionDictionary}. + *

    + * Using this constructor may cause incorrect memory estimations of the dictionary size. + */ public DictionaryEncodedColumnIndexer() { - this.dimLookup = new DimensionDictionary(); + this(new DimensionDictionary<>()); + } + + /** + * Creates a new DictionaryEncodedColumnIndexer. + * + * @param dimLookup Dimension Dictionary to lookup dimension values. + */ + public DictionaryEncodedColumnIndexer(@NotNull DimensionDictionary dimLookup) + { + this.dimLookup = Preconditions.checkNotNull(dimLookup); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java index 0cf03fca57f9..26d123e78f85 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java @@ -22,7 +22,6 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; -import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; @@ -123,9 +122,8 @@ public int add(@Nullable T originalValue) valueToId.put(originalValue, index); idToValue.add(originalValue); - long sizeOfString = getObjectSize(originalValue); - long sizeOfReference = Long.BYTES; - sizeInBytes.addAndGet(sizeOfString + 2 * sizeOfReference); + // Add size of new dim value and 2 references (valueToId and idToValue) + sizeInBytes.addAndGet(estimateSizeOfValue(originalValue) + 2 * Long.BYTES); minValue = minValue == null || minValue.compareTo(originalValue) > 0 ? originalValue : minValue; maxValue = maxValue == null || maxValue.compareTo(originalValue) < 0 ? originalValue : maxValue; @@ -174,15 +172,15 @@ public SortedDimensionDictionary sort() } } - private long getObjectSize(@Nonnull T object) + /** + * Estimates the size of a dimension value in bytes. This method is called + * only when a new dimension value is being added to the lookup. + * + * @return 0 by default + */ + public long estimateSizeOfValue(T value) { - // According to https://www.ibm.com/developerworks/java/library/j-codetoheap/index.html - // String has the following memory usuage... - // 28 bytes of data for String metadata (class pointer, flags, locks, hash, count, offset, reference to char array) - // 16 bytes of data for the char array metadata (class pointer, flags, locks, size) - // 2 bytes for every letter of the string - String val = object.toString(); - return 28 + 16 + (2L * val.length()); + return 0; } } diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java index e94726ce6fab..c166e07d2aac 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java @@ -107,27 +107,34 @@ * @param class of a single actual value * */ -public interface DimensionIndexer - , EncodedKeyComponentType, ActualType extends Comparable> +public interface DimensionIndexer< + EncodedType extends Comparable, + EncodedKeyComponentType, + ActualType extends Comparable> { /** - * Given a single row value or list of row values (for multi-valued dimensions), update any internal data structures - * with the ingested values and return the row values as an array to be used within a Row key. - * - * For example, the dictionary-encoded String-type column will return an int[] containing a dictionary ID. - * - * The value within the returned array should be encoded if applicable, i.e. as instances of EncodedType. - * - * NOTE: This function can change the internal state of the DimensionIndexer. - * - * @param dimValues Single row val to process - * - * @param reportParseExceptions - * @return An array containing an encoded representation of the input row value. + * Encodes the given row value(s) of the dimension to be used within a row key. + * It also updates the internal state of the DimensionIndexer, e.g. the dimLookup. + *

    + * For example, the dictionary-encoded String-type column will return an int[] + * containing dictionary IDs. + *

    + * + * @param dimValues Value(s) of the dimension in a row. This can + * either be a single value or a list of values + * (for multi-valued dimensions) + * @param reportParseExceptions true if parse exceptions should be reported, + * false otherwise + * @return Encoded dimension value(s) to be used as a component for the row key. + * Contains an object of the {@link EncodedKeyComponentType} and the increase + * in size of the DimensionIndexer due to any newly added dimension value. */ @Nullable - EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions); + EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent( + @Nullable Object dimValues, + boolean reportParseExceptions + ); /** * This method will be called while building an {@link IncrementalIndex} whenever a known dimension column (either diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java index 5cfd5214b982..4ee69e627d02 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java @@ -45,7 +45,7 @@ public class DoubleDimensionIndexer implements DimensionIndexer processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) + public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { if (dimValues instanceof List) { throw new UnsupportedOperationException("Numeric columns do not support multivalue rows."); @@ -54,7 +54,7 @@ public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent if (d == null) { hasNulls = NullHandling.sqlCompatible(); } - return new EncodedDimensionValue<>(d, Double.BYTES); + return new EncodedKeyComponent<>(d, Double.BYTES); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java b/processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java new file mode 100644 index 000000000000..8615d00ae358 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import javax.annotation.Nullable; + +/** + * Represents the encoded component of a row key corresponding to a single dimension. + * The row key contains a component for each dimension. + *

    + * Contains: + *

      + *
    • the encoded dimension value(s)
    • + *
    • the increase in size (in bytes) caused by adding the dimension value(s)
    • + *
    + * + * @param Encoded key component type + */ +public class EncodedKeyComponent +{ + @Nullable + private final K component; + private final long incrementalSizeBytes; + + EncodedKeyComponent(@Nullable K component, long incrementalSizeBytes) + { + this.component = component; + this.incrementalSizeBytes = incrementalSizeBytes; + } + + /** + * The encoded dimension value(s) to be used a component for a row key. + */ + @Nullable + public K getComponent() + { + return component; + } + + /** + * Increase in size (in bytes) caused by adding the dimension value(s). + */ + public long getIncrementalSizeBytes() + { + return incrementalSizeBytes; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java index 3989156e0d97..a5e598cd8041 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java @@ -45,7 +45,7 @@ public class FloatDimensionIndexer implements DimensionIndexer processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) + public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { if (dimValues instanceof List) { throw new UnsupportedOperationException("Numeric columns do not support multivalue rows."); @@ -55,7 +55,7 @@ public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent( if (f == null) { hasNulls = NullHandling.sqlCompatible(); } - return new EncodedDimensionValue<>(f, Float.BYTES); + return new EncodedKeyComponent<>(f, Float.BYTES); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java index 7456725f4275..3c0521028da8 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java @@ -46,7 +46,7 @@ public class LongDimensionIndexer implements DimensionIndexer @Nullable @Override - public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) + public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { if (dimValues instanceof List) { throw new UnsupportedOperationException("Numeric columns do not support multivalue rows."); @@ -56,7 +56,7 @@ public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent(@ if (l == null) { hasNulls = NullHandling.sqlCompatible(); } - return new EncodedDimensionValue<>(l, Long.BYTES); + return new EncodedKeyComponent<>(l, Long.BYTES); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/EncodedDimensionValue.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java similarity index 64% rename from processing/src/main/java/org/apache/druid/segment/EncodedDimensionValue.java rename to processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java index c0dcac9bee20..f15037026111 100644 --- a/processing/src/main/java/org/apache/druid/segment/EncodedDimensionValue.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java @@ -19,30 +19,16 @@ package org.apache.druid.segment; -import javax.annotation.Nullable; - /** - * @param Encoded key component type + * DimensionDictionary for String dimension values. */ -public class EncodedDimensionValue +public class StringDimensionDictionary extends DimensionDictionary { - @Nullable - private final K value; - private final long incrementalSizeBytes; - - EncodedDimensionValue(@Nullable K value, long incrementalSizeBytes) - { - this.value = value; - this.incrementalSizeBytes = incrementalSizeBytes; - } - - public K getValue() - { - return value; - } - - public long getIncrementalSizeBytes() + @Override + public long estimateSizeOfValue(String value) { - return incrementalSizeBytes; + // According to https://www.ibm.com/developerworks/java/library/j-codetoheap/index.html + // Total string size = 28B (string metadata) + 16B (char array metadata) + 2B * num letters + return 28 + 16 + (2L * value.length()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 4e09105018e6..24fc3c8bf554 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -64,13 +64,14 @@ private static String emptyToNullIfNeeded(@Nullable Object o) public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean hasBitmapIndexes, boolean hasSpatialIndexes) { + super(new StringDimensionDictionary()); this.multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling; this.hasBitmapIndexes = hasBitmapIndexes; this.hasSpatialIndexes = hasSpatialIndexes; } @Override - public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) + public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions) { final int[] encodedDimensionValues; final int oldDictSize = dimLookup.size(); @@ -126,7 +127,7 @@ public EncodedDimensionValue processRowValsToUnsortedEncodedKeyComponent( // incremental size = increase in size of dimension dict + size of encoded array long keyComponentSizeBytes = (dimLookup.sizeInBytes() - oldDictSizeInBytes) + (long) encodedDimensionValues.length * Integer.BYTES; - return new EncodedDimensionValue<>(encodedDimensionValues, keyComponentSizeBytes); + return new EncodedKeyComponent<>(encodedDimensionValues, keyComponentSizeBytes); } @Override 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 7715c4652fe3..bed191aac0e9 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 @@ -57,7 +57,7 @@ import org.apache.druid.segment.DimensionIndexer; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.DoubleColumnSelector; -import org.apache.druid.segment.EncodedDimensionValue; +import org.apache.druid.segment.EncodedKeyComponent; import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.LongColumnSelector; @@ -475,11 +475,6 @@ public ColumnCapabilities getColumnCapabilities(String columnName) * @return the number of rows in the data set after adding the InputRow. If any parse failure occurs, a {@link ParseException} is returned in {@link IncrementalIndexAddResult}. */ public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededException - { - return add(row, false); - } - - public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException { return add(row, false, true); } @@ -573,9 +568,9 @@ private IncrementalIndexRowResult toIncrementalIndexRow(InputRow row, boolean us DimensionIndexer indexer = desc.getIndexer(); Object dimsKey = null; try { - final EncodedDimensionValue encodedDimensionValue + final EncodedKeyComponent encodedDimensionValue = indexer.processRowValsToUnsortedEncodedKeyComponent(row.getRaw(dimension), true); - dimsKey = encodedDimensionValue.getValue(); + dimsKey = encodedDimensionValue.getComponent(); final long keySizeDelta = useMaxMemoryEstimates ? indexer.estimateEncodedKeyComponentSize(dimsKey) : encodedDimensionValue.getIncrementalSizeBytes(); 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 d1b3fa65e810..6419cabdb19b 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 @@ -78,6 +78,32 @@ public class OnheapIncrementalIndex extends IncrementalIndex private final long maxBytesPerRowForAggregators; protected final int maxRowCount; protected final long maxBytesInMemory; + + /** + * Flag denoting if max possible values should be used to estimate on-heap mem + * usage. + *

    + * There is one instance of Aggregator per metric per row key. + *

    + * Old Method: {@code useMaxMemoryEstimates = true} (default) + *

      + *
    • Aggregator: For a given metric, compute the max memory an aggregator + * can use and multiply that by number of aggregators (same as number of + * aggregated rows or number of unique row keys)
    • + *
    • DimensionIndexer: For each row, encode dimension values and estimate + * size of original dimension values
    • + *
    + * + * New Method: {@code useMaxMemoryEstimates = false} + *
      + *
    • Aggregator: Get the initialize of an Aggregator instance, and add the + * incremental size required in each aggregation step.
    • + *
    • DimensionIndexer: For each row, encode dimension values and estimate + * size of dimension values only if they are newly added to the dictionary
    • + *
    + *

    + * Thus the new method eliminates over-estimations. + */ private final boolean useMaxMemoryEstimates; @Nullable @@ -106,6 +132,8 @@ public class OnheapIncrementalIndex extends IncrementalIndex } /** + * Old method of memory estimation. Used only when {@link #useMaxMemoryEstimates} is true. + * * Gives estimated max size per aggregator. It is assumed that every aggregator will have enough overhead for its own * object header and for a pointer to a selector. We are adding a overhead-factor for each object as additional 16 * bytes. @@ -251,6 +279,8 @@ private long factorizeAggs( { long totalInitialSizeBytes = 0L; rowContainer.set(row); + + final long aggReferenceSize = Long.BYTES; for (int i = 0; i < metrics.length; i++) { final AggregatorFactory agg = metrics[i]; @@ -261,6 +291,7 @@ private long factorizeAggs( agg.factorizeWithSize(selectors.get(agg.getName())); aggs[i] = aggregatorAndSize.getAggregator(); totalInitialSizeBytes += aggregatorAndSize.getInitialSizeBytes(); + totalInitialSizeBytes += aggReferenceSize; } } rowContainer.set(null); diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java index 13c6a6834916..b33d16b07ddd 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexBenchmark.java @@ -124,7 +124,7 @@ public MapIncrementalIndex( sortFacts, maxRowCount, maxBytesInMemory, - false + true ); } @@ -147,7 +147,7 @@ public MapIncrementalIndex( true, maxRowCount, maxBytesInMemory, - false + true ); } From 919ad0ea0a352fc26a05dab39ea0f697b703f254 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 17 Jan 2022 14:17:28 +0530 Subject: [PATCH 05/15] Separate old and new flow based on useMaxMemoryEstimates --- .../DictionaryEncodedColumnIndexer.java | 2 +- .../druid/segment/DimensionDictionary.java | 35 +++++++++++++--- .../druid/segment/DimensionHandler.java | 5 ++- .../druid/segment/DimensionIndexer.java | 13 +----- .../druid/segment/DoubleDimensionHandler.java | 2 +- .../druid/segment/DoubleDimensionIndexer.java | 6 --- .../druid/segment/EncodedKeyComponent.java | 31 ++++++++------ .../druid/segment/FloatDimensionHandler.java | 2 +- .../druid/segment/FloatDimensionIndexer.java | 6 --- .../druid/segment/LongDimensionHandler.java | 2 +- .../druid/segment/LongDimensionIndexer.java | 6 --- .../segment/StringDimensionDictionary.java | 8 ++++ .../druid/segment/StringDimensionHandler.java | 4 +- .../druid/segment/StringDimensionIndexer.java | 31 ++++++++++---- .../segment/incremental/IncrementalIndex.java | 40 +++++++++---------- .../incremental/OnheapIncrementalIndex.java | 10 +++++ .../druid/segment/realtime/plumber/Sink.java | 2 +- 17 files changed, 121 insertions(+), 84 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java index f78ea9599897..a2cbfb76e818 100644 --- a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java @@ -49,7 +49,7 @@ public abstract class DictionaryEncodedColumnIndexer - * Using this constructor may cause incorrect memory estimations of the dictionary size. + * Using this constructor disables memory estimations of the dictionary size. */ public DictionaryEncodedColumnIndexer() { diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java index 26d123e78f85..1c2c4aa1f8e9 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java @@ -46,15 +46,28 @@ public class DimensionDictionary> private volatile int idForNull = ABSENT_VALUE_ID; private final AtomicLong sizeInBytes = new AtomicLong(0L); + private final boolean computeOnheapSize; + private final Object2IntMap valueToId = new Object2IntOpenHashMap<>(); private final List idToValue = new ArrayList<>(); private final ReentrantReadWriteLock lock; public DimensionDictionary() + { + this(false); + } + + /** + * Creates a DimensionDictionary. + * + * @param computeOnheapSize true if on-heap size of the dictionary should be maintained, false otherwise + */ + public DimensionDictionary(boolean computeOnheapSize) { this.lock = new ReentrantReadWriteLock(); valueToId.defaultReturnValue(ABSENT_VALUE_ID); + this.computeOnheapSize = computeOnheapSize; } public int getId(@Nullable T value) @@ -98,8 +111,17 @@ public int size() } } + /** + * Gets the current size of this dictionary in bytes. + * + * @throws IllegalStateException if size computation is disabled. + */ public long sizeInBytes() { + if (!computeOnheapSize) { + throw new IllegalStateException("On-heap size computation is disabled"); + } + return sizeInBytes.get(); } @@ -122,8 +144,10 @@ public int add(@Nullable T originalValue) valueToId.put(originalValue, index); idToValue.add(originalValue); - // Add size of new dim value and 2 references (valueToId and idToValue) - sizeInBytes.addAndGet(estimateSizeOfValue(originalValue) + 2 * Long.BYTES); + if (computeOnheapSize) { + // Add size of new dim value and 2 references (valueToId and idToValue) + sizeInBytes.addAndGet(estimateSizeOfValue(originalValue) + 2L * Long.BYTES); + } minValue = minValue == null || minValue.compareTo(originalValue) > 0 ? originalValue : minValue; maxValue = maxValue == null || maxValue.compareTo(originalValue) < 0 ? originalValue : maxValue; @@ -173,14 +197,15 @@ public SortedDimensionDictionary sort() } /** - * Estimates the size of a dimension value in bytes. This method is called + * Estimates the size of the dimension value in bytes. This method is called * only when a new dimension value is being added to the lookup. * - * @return 0 by default + * @throws UnsupportedOperationException Implementations that want to estimate + * memory must override this method. */ public long estimateSizeOfValue(T value) { - return 0; + throw new UnsupportedOperationException(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java index adfa6c6d5d25..853850625cbc 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandler.java @@ -29,7 +29,6 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import javax.annotation.Nullable; - import java.util.Comparator; /** @@ -95,9 +94,11 @@ default MultiValueHandling getMultivalueHandling() * Creates a new DimensionIndexer, a per-dimension object responsible for processing ingested rows in-memory, used * by the IncrementalIndex. See {@link DimensionIndexer} interface for more information. * + * @param useMaxMemoryEstimates true if the created DimensionIndexer should use + * maximum values to estimate on-heap memory * @return A new DimensionIndexer object. */ - DimensionIndexer makeIndexer(); + DimensionIndexer makeIndexer(boolean useMaxMemoryEstimates); /** * Creates a new DimensionMergerV9, a per-dimension object responsible for merging indexes/row data across segments diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java index c166e07d2aac..6917035c8374 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java @@ -127,8 +127,8 @@ public interface DimensionIndexer< * @param reportParseExceptions true if parse exceptions should be reported, * false otherwise * @return Encoded dimension value(s) to be used as a component for the row key. - * Contains an object of the {@link EncodedKeyComponentType} and the increase - * in size of the DimensionIndexer due to any newly added dimension value. + * Contains an object of the {@link EncodedKeyComponentType} and the effective + * size of the key component in bytes. */ @Nullable EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent( @@ -149,15 +149,6 @@ EncodedKeyComponent processRowValsToUnsortedEncodedKeyC */ void setSparseIndexed(); - /** - * Gives the estimated size in bytes for the given key - * - * @param key dimension value array from a TimeAndDims key - * - * @return the estimated size in bytes of the key - */ - long estimateEncodedKeyComponentSize(EncodedKeyComponentType key); - /** * Given an encoded value that was ordered by associated actual value, return the equivalent * encoded value ordered by time of ingestion. diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java index 928770379750..9fbc9436be6d 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionHandler.java @@ -70,7 +70,7 @@ public DimensionSchema getDimensionSchema(ColumnCapabilities capabilities) } @Override - public DimensionIndexer makeIndexer() + public DimensionIndexer makeIndexer(boolean useMaxMemoryEstimates) { return new DoubleDimensionIndexer(); } diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java index 4ee69e627d02..d9568696f2db 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java @@ -63,12 +63,6 @@ public void setSparseIndexed() hasNulls = NullHandling.sqlCompatible(); } - @Override - public long estimateEncodedKeyComponentSize(Double key) - { - return Double.BYTES; - } - @Override public Double getUnsortedEncodedValueFromSorted(Double sortedIntermediateValue) { diff --git a/processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java b/processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java index 8615d00ae358..abe9449a9587 100644 --- a/processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java +++ b/processing/src/main/java/org/apache/druid/segment/EncodedKeyComponent.java @@ -24,12 +24,6 @@ /** * Represents the encoded component of a row key corresponding to a single dimension. * The row key contains a component for each dimension. - *

    - * Contains: - *

      - *
    • the encoded dimension value(s)
    • - *
    • the increase in size (in bytes) caused by adding the dimension value(s)
    • - *
    * * @param Encoded key component type */ @@ -37,16 +31,25 @@ public class EncodedKeyComponent { @Nullable private final K component; - private final long incrementalSizeBytes; + private final long effectiveSizeBytes; - EncodedKeyComponent(@Nullable K component, long incrementalSizeBytes) + /** + * Creates an EncodedKeyComponent corresponding to a single dimension. + * + * @param component The encoded dimension value(s) + * @param effectiveSizeBytes Effective size of the key component in bytes. This + * value is used to estimate on-heap memory usage and + * must account for the footprint of both the original + * and encoded dimension values, as applicable. + */ + EncodedKeyComponent(@Nullable K component, long effectiveSizeBytes) { this.component = component; - this.incrementalSizeBytes = incrementalSizeBytes; + this.effectiveSizeBytes = effectiveSizeBytes; } /** - * The encoded dimension value(s) to be used a component for a row key. + * Encoded dimension value(s) to be used as a component for a row key. */ @Nullable public K getComponent() @@ -55,10 +58,12 @@ public K getComponent() } /** - * Increase in size (in bytes) caused by adding the dimension value(s). + * Effective size of the key component in bytes. This value is used to estimate + * on-heap memory usage and accounts for the memory footprint of both the + * original and encoded dimension values, as applicable. */ - public long getIncrementalSizeBytes() + public long getEffectiveSizeBytes() { - return incrementalSizeBytes; + return effectiveSizeBytes; } } diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java index c689bbffb6a1..4763b0e8be12 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionHandler.java @@ -70,7 +70,7 @@ public DimensionSchema getDimensionSchema(ColumnCapabilities capabilities) } @Override - public DimensionIndexer makeIndexer() + public DimensionIndexer makeIndexer(boolean useMaxMemoryEstimates) { return new FloatDimensionIndexer(); } diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java index a5e598cd8041..66e412b47b43 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java @@ -64,12 +64,6 @@ public void setSparseIndexed() hasNulls = NullHandling.sqlCompatible(); } - @Override - public long estimateEncodedKeyComponentSize(Float key) - { - return Float.BYTES; - } - @Override public Float getUnsortedEncodedValueFromSorted(Float sortedIntermediateValue) { diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java index a5b602b063db..51e13396840c 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionHandler.java @@ -70,7 +70,7 @@ public DimensionSchema getDimensionSchema(ColumnCapabilities capabilities) } @Override - public DimensionIndexer makeIndexer() + public DimensionIndexer makeIndexer(boolean useMaxMemoryEstimates) { return new LongDimensionIndexer(); } diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java index 3c0521028da8..d0dd84f573de 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java @@ -65,12 +65,6 @@ public void setSparseIndexed() hasNulls = NullHandling.sqlCompatible(); } - @Override - public long estimateEncodedKeyComponentSize(Long key) - { - return Long.BYTES; - } - @Override public Long getUnsortedEncodedValueFromSorted(Long sortedIntermediateValue) { diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java index f15037026111..5e9f3b8efcbf 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java @@ -21,9 +21,17 @@ /** * DimensionDictionary for String dimension values. + *

    + * With StringDimensionDictionary, on-heap memory estimation of the dictionary + * size is always enabled. */ public class StringDimensionDictionary extends DimensionDictionary { + public StringDimensionDictionary() + { + super(true); + } + @Override public long estimateSizeOfValue(String value) { diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java index b125eb1d098f..237b4745335f 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionHandler.java @@ -147,9 +147,9 @@ public SettableColumnValueSelector makeNewSettableEncodedValueSelector() } @Override - public DimensionIndexer makeIndexer() + public DimensionIndexer makeIndexer(boolean useMaxMemoryEstimates) { - return new StringDimensionIndexer(multiValueHandling, hasBitmapIndexes, hasSpatialIndexes); + return new StringDimensionIndexer(multiValueHandling, hasBitmapIndexes, hasSpatialIndexes, useMaxMemoryEstimates); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 24fc3c8bf554..7b0c179ec066 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -60,14 +60,21 @@ private static String emptyToNullIfNeeded(@Nullable Object o) private final MultiValueHandling multiValueHandling; private final boolean hasBitmapIndexes; private final boolean hasSpatialIndexes; + private final boolean useMaxMemoryEstimates; private volatile boolean hasMultipleValues = false; - public StringDimensionIndexer(MultiValueHandling multiValueHandling, boolean hasBitmapIndexes, boolean hasSpatialIndexes) + public StringDimensionIndexer( + MultiValueHandling multiValueHandling, + boolean hasBitmapIndexes, + boolean hasSpatialIndexes, + boolean useMaxMemoryEstimates + ) { - super(new StringDimensionDictionary()); + super(useMaxMemoryEstimates ? new DimensionDictionary<>() : new StringDimensionDictionary()); this.multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling; this.hasBitmapIndexes = hasBitmapIndexes; this.hasSpatialIndexes = hasSpatialIndexes; + this.useMaxMemoryEstimates = useMaxMemoryEstimates; } @Override @@ -75,7 +82,7 @@ public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@N { final int[] encodedDimensionValues; final int oldDictSize = dimLookup.size(); - final long oldDictSizeInBytes = dimLookup.sizeInBytes(); + final long oldDictSizeInBytes = useMaxMemoryEstimates ? 0 : dimLookup.sizeInBytes(); if (dimValues == null) { final int nullId = dimLookup.getId(null); @@ -125,13 +132,23 @@ public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@N } // incremental size = increase in size of dimension dict + size of encoded array - long keyComponentSizeBytes = (dimLookup.sizeInBytes() - oldDictSizeInBytes) - + (long) encodedDimensionValues.length * Integer.BYTES; + long keyComponentSizeBytes; + if (useMaxMemoryEstimates) { + keyComponentSizeBytes = estimateEncodedKeyComponentSize(encodedDimensionValues); + } else { + keyComponentSizeBytes = (dimLookup.sizeInBytes() - oldDictSizeInBytes) + + (long) encodedDimensionValues.length * Integer.BYTES; + } return new EncodedKeyComponent<>(encodedDimensionValues, keyComponentSizeBytes); } - @Override - public long estimateEncodedKeyComponentSize(int[] key) + /** + * Estimates size of the given key component. + *

    + * Deprecated method. Use {@link #processRowValsToUnsortedEncodedKeyComponent(Object, boolean)} + * and {@link EncodedKeyComponent#getEffectiveSizeBytes()}. + */ + long estimateEncodedKeyComponentSize(int[] key) { // string length is being accounted for each time they are referenced, based on dimension handler interface, // even though they are stored just once. It may overestimate the size by a bit, but we wanted to leave 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 bed191aac0e9..703fdd503a2c 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 @@ -476,7 +476,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) */ public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededException { - return add(row, false, true); + return add(row, false); } /** @@ -489,17 +489,13 @@ public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededExcep * * @param row the row of data to add * @param skipMaxRowsInMemoryCheck whether or not to skip the check of rows exceeding the max rows limit - * @param useMaxMemoryEstimates whether or not to use maximum values to estimate memory * @return the number of rows in the data set after adding the InputRow. If any parse failure occurs, a {@link ParseException} is returned in {@link IncrementalIndexAddResult}. * @throws IndexSizeExceededException this exception is thrown once it reaches max rows limit and skipMaxRowsInMemoryCheck is set to false. */ - public IncrementalIndexAddResult add( - InputRow row, - boolean skipMaxRowsInMemoryCheck, - boolean useMaxMemoryEstimates - ) throws IndexSizeExceededException + public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) + throws IndexSizeExceededException { - IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row, useMaxMemoryEstimates); + IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row); final AddToFactsResult addToFactsResult = addToFacts( row, incrementalIndexRowResult.getIncrementalIndexRow(), @@ -522,11 +518,6 @@ public IncrementalIndexAddResult add( @VisibleForTesting IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) - { - return toIncrementalIndexRow(row, true); - } - - private IncrementalIndexRowResult toIncrementalIndexRow(InputRow row, boolean useMaxMemoryEstimates) { row = formatRow(row); if (row.getTimestampFromEpoch() < minTimestamp) { @@ -568,13 +559,10 @@ private IncrementalIndexRowResult toIncrementalIndexRow(InputRow row, boolean us DimensionIndexer indexer = desc.getIndexer(); Object dimsKey = null; try { - final EncodedKeyComponent encodedDimensionValue + final EncodedKeyComponent encodedKeyComponent = indexer.processRowValsToUnsortedEncodedKeyComponent(row.getRaw(dimension), true); - dimsKey = encodedDimensionValue.getComponent(); - final long keySizeDelta = useMaxMemoryEstimates - ? indexer.estimateEncodedKeyComponentSize(dimsKey) - : encodedDimensionValue.getIncrementalSizeBytes(); - dimsKeySize += keySizeDelta; + dimsKey = encodedKeyComponent.getComponent(); + dimsKeySize += encodedKeyComponent.getEffectiveSizeBytes(); } catch (ParseException pe) { parseExceptionMessages.add(pe.getMessage()); @@ -879,12 +867,17 @@ public void loadDimensionIterable( @GuardedBy("dimensionDescs") private DimensionDesc addNewDimension(String dim, DimensionHandler handler) { - DimensionDesc desc = new DimensionDesc(dimensionDescs.size(), dim, handler); + DimensionDesc desc = initDimension(dimensionDescs.size(), dim, handler); dimensionDescs.put(dim, desc); dimensionDescsList.add(desc); return desc; } + protected DimensionDesc initDimension(int dimensionIndex, String dimensionName, DimensionHandler dimensionHandler) + { + return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler); + } + public List getMetricNames() { return ImmutableList.copyOf(metricDescs.keySet()); @@ -942,11 +935,16 @@ public static final class DimensionDesc private final DimensionIndexer indexer; public DimensionDesc(int index, String name, DimensionHandler handler) + { + this(index, name, handler, true); + } + + public DimensionDesc(int index, String name, DimensionHandler handler, boolean useMaxMemoryEstimates) { this.index = index; this.name = name; this.handler = handler; - this.indexer = handler.makeIndexer(); + this.indexer = handler.makeIndexer(useMaxMemoryEstimates); } public int getIndex() 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 6419cabdb19b..53a24f4b3bc4 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 @@ -193,6 +193,16 @@ protected void initAggs( } } + @Override + protected DimensionDesc initDimension( + int dimensionIndex, + String dimensionName, + DimensionHandler dimensionHandler + ) + { + return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler, useMaxMemoryEstimates); + } + @Override protected AddToFactsResult addToFacts( InputRow row, 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 656347920de7..883d6f5a58b2 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 @@ -181,7 +181,7 @@ public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCh return Plumber.DUPLICATE; } - return index.add(row, skipMaxRowsInMemoryCheck, useMaxMemoryEstimates); + return index.add(row, skipMaxRowsInMemoryCheck); } } From 766abca1c7a6f431c88b3310b063f69104904ec8 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Mon, 17 Jan 2022 22:21:06 +0530 Subject: [PATCH 06/15] Fix InputSourceSampler error --- .../druid/indexing/overlord/sampler/InputSourceSampler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c864658a2610..21685c4d7455 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 @@ -154,7 +154,7 @@ public SamplerResponse sample( //keep the index of the row to be added to responseRows for further use final int rowIndex = responseRows.size(); - IncrementalIndexAddResult addResult = index.add(new SamplerInputRow(row, rowIndex), true, true); + IncrementalIndexAddResult addResult = index.add(new SamplerInputRow(row, rowIndex), true); if (addResult.hasParseException()) { responseRows.add(new SamplerResponseRow( rawColumns, From 44d4cb48e7de62fde5da86fd5c586537f95ec2fa Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 18 Jan 2022 01:48:48 +0530 Subject: [PATCH 07/15] Fix StringDimensionIndexerBenchmark --- .../benchmark/indexing/StringDimensionIndexerBenchmark.java | 2 +- .../java/org/apache/druid/segment/StringDimensionIndexer.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/StringDimensionIndexerBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/StringDimensionIndexerBenchmark.java index 2e4490bd26c3..487c7ac0b90d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/StringDimensionIndexerBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/StringDimensionIndexerBenchmark.java @@ -59,7 +59,7 @@ public class StringDimensionIndexerBenchmark @Setup public void setup() { - indexer = new StringDimensionIndexer(DimensionSchema.MultiValueHandling.ofDefault(), true, false); + indexer = new StringDimensionIndexer(DimensionSchema.MultiValueHandling.ofDefault(), true, false, true); for (int i = 0; i < cardinality; i++) { indexer.processRowValsToUnsortedEncodedKeyComponent("abcd-" + i, true); diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 7b0c179ec066..877434ce4c9d 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -148,7 +148,7 @@ public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@N * Deprecated method. Use {@link #processRowValsToUnsortedEncodedKeyComponent(Object, boolean)} * and {@link EncodedKeyComponent#getEffectiveSizeBytes()}. */ - long estimateEncodedKeyComponentSize(int[] key) + public long estimateEncodedKeyComponentSize(int[] key) { // string length is being accounted for each time they are referenced, based on dimension handler interface, // even though they are stored just once. It may overestimate the size by a bit, but we wanted to leave From 3ce9ed2d910914af2aaafa53d2f038c458a201e3 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 18 Jan 2022 10:54:23 +0530 Subject: [PATCH 08/15] Add test for SketchAggregator.aggregateWithSize() --- .../theta/SketchAggregationTest.java | 25 +++++++++++++++++ .../theta/SketchAggregatorFactoryTest.java | 4 +-- .../druid/segment/DimensionDictionary.java | 28 +++++++++---------- .../segment/StringDimensionDictionary.java | 11 ++++---- 4 files changed, 46 insertions(+), 22 deletions(-) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index 054693f27679..e4c4f573756c 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -546,6 +546,31 @@ public void testUpdateUnionWithDouble() Assert.assertEquals(1, ((SketchHolder) agg.get()).getSketch().getEstimate(), 0); } + @Test + public void testAggregateWithSize() + { + final String[] columnValues = new String[]{"a", "b", "c", "a"}; + final TestObjectColumnSelector selector = new TestObjectColumnSelector<>(columnValues); + final SketchAggregator agg = new SketchAggregator(selector, 16); + + // Verify initial size of sketch + Assert.assertEquals(48L, agg.getInitialSizeBytes()); + Assert.assertEquals(328L, agg.aggregateWithSize()); + + // Verify that subsequent size deltas are zero + for (int i = 1; i < columnValues.length; ++i) { + selector.increment(); + long sizeDelta = agg.aggregateWithSize(); + Assert.assertEquals(0, sizeDelta); + } + + // Verify unique count estimate + SketchHolder sketchHolder = (SketchHolder) agg.get(); + Assert.assertEquals(3, sketchHolder.getEstimate(), 0); + Assert.assertNotNull(sketchHolder.getSketch()); + Assert.assertEquals(3, sketchHolder.getSketch().getEstimate(), 0); + } + private void assertPostAggregatorSerde(PostAggregator agg) throws Exception { Assert.assertEquals( diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java index c8fb91ca7a97..1762ef3b4849 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java @@ -74,10 +74,10 @@ public void testFactorizeSized() EasyMock.replay(colSelectorFactory); AggregatorAndSize aggregatorAndSize = AGGREGATOR_16384.factorizeWithSize(colSelectorFactory); - Assert.assertEquals(28, aggregatorAndSize.getInitialSizeBytes()); + Assert.assertEquals(48, aggregatorAndSize.getInitialSizeBytes()); aggregatorAndSize = AGGREGATOR_32768.factorizeWithSize(colSelectorFactory); - Assert.assertEquals(28, aggregatorAndSize.getInitialSizeBytes()); + Assert.assertEquals(48, aggregatorAndSize.getInitialSizeBytes()); } @Test diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java index 1c2c4aa1f8e9..17c2cc4ce1d6 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionDictionary.java @@ -46,28 +46,15 @@ public class DimensionDictionary> private volatile int idForNull = ABSENT_VALUE_ID; private final AtomicLong sizeInBytes = new AtomicLong(0L); - private final boolean computeOnheapSize; - private final Object2IntMap valueToId = new Object2IntOpenHashMap<>(); private final List idToValue = new ArrayList<>(); private final ReentrantReadWriteLock lock; public DimensionDictionary() - { - this(false); - } - - /** - * Creates a DimensionDictionary. - * - * @param computeOnheapSize true if on-heap size of the dictionary should be maintained, false otherwise - */ - public DimensionDictionary(boolean computeOnheapSize) { this.lock = new ReentrantReadWriteLock(); valueToId.defaultReturnValue(ABSENT_VALUE_ID); - this.computeOnheapSize = computeOnheapSize; } public int getId(@Nullable T value) @@ -118,7 +105,7 @@ public int size() */ public long sizeInBytes() { - if (!computeOnheapSize) { + if (!computeOnHeapSize()) { throw new IllegalStateException("On-heap size computation is disabled"); } @@ -144,7 +131,7 @@ public int add(@Nullable T originalValue) valueToId.put(originalValue, index); idToValue.add(originalValue); - if (computeOnheapSize) { + if (computeOnHeapSize()) { // Add size of new dim value and 2 references (valueToId and idToValue) sizeInBytes.addAndGet(estimateSizeOfValue(originalValue) + 2L * Long.BYTES); } @@ -208,4 +195,15 @@ public long estimateSizeOfValue(T value) throw new UnsupportedOperationException(); } + /** + * Whether on-heap size of this dictionary should be computed. + * + * @return false, by default. Implementations that want to estimate memory + * must override this method. + */ + public boolean computeOnHeapSize() + { + return false; + } + } diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java index 5e9f3b8efcbf..3072f6d123c1 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java @@ -27,11 +27,6 @@ */ public class StringDimensionDictionary extends DimensionDictionary { - public StringDimensionDictionary() - { - super(true); - } - @Override public long estimateSizeOfValue(String value) { @@ -39,4 +34,10 @@ public long estimateSizeOfValue(String value) // Total string size = 28B (string metadata) + 16B (char array metadata) + 2B * num letters return 28 + 16 + (2L * value.length()); } + + @Override + public boolean computeOnHeapSize() + { + return true; + } } From ba131ab65a9e3cf46ebd7985eb603629157a3ec0 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 20 Jan 2022 00:08:48 +0530 Subject: [PATCH 09/15] Add StringDimensionIndexerTest --- .../theta/SketchAggregationTest.java | 25 +- .../druid/segment/StringDimensionIndexer.java | 12 +- .../segment/StringDimensionIndexerTest.java | 229 ++++++++++++++++++ 3 files changed, 255 insertions(+), 11 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index e4c4f573756c..fb36a0797657 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -549,16 +549,31 @@ public void testUpdateUnionWithDouble() @Test public void testAggregateWithSize() { - final String[] columnValues = new String[]{"a", "b", "c", "a"}; + final String[] columnValues = new String[20]; + for (int i = 0; i < columnValues.length; ++i) { + columnValues[i] = "" + i; + } + final TestObjectColumnSelector selector = new TestObjectColumnSelector<>(columnValues); - final SketchAggregator agg = new SketchAggregator(selector, 16); + final SketchAggregator agg = new SketchAggregator(selector, 128); // Verify initial size of sketch Assert.assertEquals(48L, agg.getInitialSizeBytes()); Assert.assertEquals(328L, agg.aggregateWithSize()); // Verify that subsequent size deltas are zero - for (int i = 1; i < columnValues.length; ++i) { + for (int i = 1; i < 16; ++i) { + selector.increment(); + long sizeDelta = agg.aggregateWithSize(); + Assert.assertEquals(0, sizeDelta); + } + + // Verify that size delta is positive when sketch resizes + selector.increment(); + long deltaAtResize = agg.aggregateWithSize(); + Assert.assertEquals(1792, deltaAtResize); + + for (int i = 17; i < columnValues.length; ++i) { selector.increment(); long sizeDelta = agg.aggregateWithSize(); Assert.assertEquals(0, sizeDelta); @@ -566,9 +581,9 @@ public void testAggregateWithSize() // Verify unique count estimate SketchHolder sketchHolder = (SketchHolder) agg.get(); - Assert.assertEquals(3, sketchHolder.getEstimate(), 0); + Assert.assertEquals(columnValues.length, sketchHolder.getEstimate(), 0); Assert.assertNotNull(sketchHolder.getSketch()); - Assert.assertEquals(3, sketchHolder.getSketch().getEstimate(), 0); + Assert.assertEquals(columnValues.length, sketchHolder.getSketch().getEstimate(), 0); } private void assertPostAggregatorSerde(PostAggregator agg) throws Exception diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 877434ce4c9d..7837253b93c6 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -131,15 +131,15 @@ public EncodedKeyComponent processRowValsToUnsortedEncodedKeyComponent(@N sortedLookup = null; } - // incremental size = increase in size of dimension dict + size of encoded array - long keyComponentSizeBytes; + long effectiveSizeBytes; if (useMaxMemoryEstimates) { - keyComponentSizeBytes = estimateEncodedKeyComponentSize(encodedDimensionValues); + effectiveSizeBytes = estimateEncodedKeyComponentSize(encodedDimensionValues); } else { - keyComponentSizeBytes = (dimLookup.sizeInBytes() - oldDictSizeInBytes) - + (long) encodedDimensionValues.length * Integer.BYTES; + // size of encoded array + dictionary size change + effectiveSizeBytes = 16L + (long) encodedDimensionValues.length * Integer.BYTES + + (dimLookup.sizeInBytes() - oldDictSizeInBytes); } - return new EncodedKeyComponent<>(encodedDimensionValues, keyComponentSizeBytes); + return new EncodedKeyComponent<>(encodedDimensionValues, effectiveSizeBytes); } /** diff --git a/processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java new file mode 100644 index 000000000000..949a8408d9ff --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java @@ -0,0 +1,229 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; + +/** + * Unit tests for {@link StringDimensionIndexer}. + */ +public class StringDimensionIndexerTest +{ + @Before + public void setup() + { + NullHandling.initializeForTests(); + } + + @Test + public void testProcessRowValsToEncodedKeyComponent_usingAvgEstimates() + { + final StringDimensionIndexer indexer = new StringDimensionIndexer( + DimensionSchema.MultiValueHandling.SORTED_ARRAY, + true, + false, + false + ); + + long totalEstimatedSize = 0L; + + // Verify size for a non-empty dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + "abc", + new int[]{0}, + 86L + ); + + // Verify size for null dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + null, + new int[]{1}, + 20L + ); + + // Verify size delta with repeated dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + "abc", + new int[]{0}, + 20L + ); + // Verify size delta with newly added dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + "def", + new int[]{2}, + 86L + ); + + // Verify size delta for multi-values + totalEstimatedSize += verifyEncodedValues( + indexer, + Arrays.asList("abc", "def", "ghi"), + new int[]{0, 2, 3}, + 94L + ); + + Assert.assertEquals(306L, totalEstimatedSize); + } + + @Test + public void testProcessRowValsToEncodedKeyComponent_usingMaxEstimates() + { + final StringDimensionIndexer indexer = new StringDimensionIndexer( + DimensionSchema.MultiValueHandling.SORTED_ARRAY, + true, + false, + true + ); + + long totalEstimatedSize = 0L; + + // Verify size for a non-empty dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + "abc", + new int[]{0}, + 54L + ); + + // Verify size for null dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + null, + new int[]{1}, + 4L + ); + + // Verify size delta with repeated dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + "abc", + new int[]{0}, + 54L + ); + // Verify size delta with newly added dimension value + totalEstimatedSize += verifyEncodedValues( + indexer, + "def", + new int[]{2}, + 54L + ); + + // Verify size delta for multi-values + totalEstimatedSize += verifyEncodedValues( + indexer, + Arrays.asList("abc", "def", "ghi"), + new int[]{0, 2, 3}, + 162L + ); + + Assert.assertEquals(328L, totalEstimatedSize); + } + + @Test + public void testProcessRowValsToEncodedKeyComponent_comparison() + { + // Create indexers with useMaxMemoryEstimates = true/false + final StringDimensionIndexer indexerForAvgEstimates = new StringDimensionIndexer( + DimensionSchema.MultiValueHandling.SORTED_ARRAY, + true, + false, + false + ); + StringDimensionIndexer indexerForMaxEstimates = new StringDimensionIndexer( + DimensionSchema.MultiValueHandling.SORTED_ARRAY, + true, + false, + true + ); + + // Verify sizes with newly added dimension values + long totalSizeWithMaxEstimates = 0L; + long totalSizeWithAvgEstimates = 0L; + for (int i = 0; i < 10; ++i) { + final String dimValue = "value-" + i; + totalSizeWithMaxEstimates += verifyEncodedValues( + indexerForMaxEstimates, + dimValue, + new int[]{i}, + 62L + ); + totalSizeWithAvgEstimates += verifyEncodedValues( + indexerForAvgEstimates, + dimValue, + new int[]{i}, + 94L + ); + } + + // If all dimension values are unique (or cardinality is high), + // estimates with useMaxMemoryEstimates = false tend to be higher + Assert.assertEquals(620L, totalSizeWithMaxEstimates); + Assert.assertEquals(940L, totalSizeWithAvgEstimates); + + // Verify sizes with repeated dimension values + for (int i = 0; i < 100; ++i) { + final int index = i % 10; + final String dimValue = "value-" + index; + totalSizeWithMaxEstimates += verifyEncodedValues( + indexerForMaxEstimates, + dimValue, + new int[]{index}, + 62L + ); + totalSizeWithAvgEstimates += verifyEncodedValues( + indexerForAvgEstimates, + dimValue, + new int[]{index}, + 20L + ); + } + + // If dimension values are frequently repeated (cardinality is low), + // estimates with useMaxMemoryEstimates = false tend to be much lower + Assert.assertEquals(6820L, totalSizeWithMaxEstimates); + Assert.assertEquals(2940L, totalSizeWithAvgEstimates); + } + + private long verifyEncodedValues( + StringDimensionIndexer indexer, + Object dimensionValues, + int[] expectedEncodedValues, + long expectedSizeDelta + ) + { + EncodedKeyComponent encodedKeyComponent = indexer + .processRowValsToUnsortedEncodedKeyComponent(dimensionValues, false); + Assert.assertArrayEquals(expectedEncodedValues, encodedKeyComponent.getComponent()); + Assert.assertEquals(expectedSizeDelta, encodedKeyComponent.getEffectiveSizeBytes()); + + return encodedKeyComponent.getEffectiveSizeBytes(); + } + +} \ No newline at end of file From a83316fed763d8d823cd492da4b53a55f569a7ce Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 20 Jan 2022 10:00:16 +0530 Subject: [PATCH 10/15] Fix coverage and tests --- .../incremental/IncrementalIndexRow.java | 22 ++++++++---- .../mean/DoubleMeanAggregationTest.java | 35 +++++++++++++++++++ .../mean/DoubleMeanAggregatorFactoryTest.java | 18 ++++++++++ .../segment/StringDimensionIndexerTest.java | 13 ++----- .../IncrementalIndexRowSizeTest.java | 1 + 5 files changed, 73 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index 8ca1feb4365f..987ee5f8bf7c 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java @@ -19,10 +19,12 @@ package org.apache.druid.segment.incremental; +import com.google.common.base.Function; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.DimensionIndexer; +import javax.annotation.Nullable; import java.lang.reflect.Array; import java.util.Arrays; import java.util.Collections; @@ -111,7 +113,8 @@ void setRowIndex(int rowIndex) } /** - * Estimates the size of IncrementalIndexRow key by adding the following components: + * bytesInMemory estimates the size of IncrementalIndexRow key, it takes into account the timestamp(long), + * dims(Object Array) and dimensionDescsList(List). Each of these are calculated as follows: *

      *
    • timestamp : Long.BYTES *
    • dims array : Integer.BYTES * array length + Long.BYTES (dims object) + dimsKeySize(passed via constructor) @@ -123,7 +126,7 @@ void setRowIndex(int rowIndex) */ public long estimateBytesInMemory() { - long sizeInBytes = Long.BYTES + Integer.BYTES * (long) dims.length + Long.BYTES + Long.BYTES; + long sizeInBytes = Long.BYTES + ((long) Integer.BYTES) * dims.length + Long.BYTES + Long.BYTES; sizeInBytes += dimsKeySize; return sizeInBytes; } @@ -134,10 +137,17 @@ public String toString() return "IncrementalIndexRow{" + "timestamp=" + DateTimes.utc(timestamp) + ", dims=" + Lists.transform( - Arrays.asList(dims), - input -> (input == null || Array.getLength(input) == 0) - ? Collections.singletonList("null") - : Collections.singletonList(input) + Arrays.asList(dims), new Function() + { + @Override + public Object apply(@Nullable Object input) + { + if (input == null || Array.getLength(input) == 0) { + return Collections.singletonList("null"); + } + return Collections.singletonList(input); + } + } ) + '}'; } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java index 746fdbdc0dcf..74846b566a46 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java @@ -34,15 +34,20 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregationTestHelper; +import org.apache.druid.query.aggregation.Aggregator; +import org.apache.druid.query.aggregation.AggregatorAndSize; +import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.Segment; import org.apache.druid.timeline.SegmentId; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -173,4 +178,34 @@ public void testAggretatorUsingTimeseriesQuery(boolean doVectorize) throws Excep Assert.assertEquals(6.2d, result.getDoubleMetric("meanOnString").doubleValue(), 0.0001d); Assert.assertEquals(4.1333d, result.getDoubleMetric("meanOnMultiValue").doubleValue(), 0.0001d); } + + @Test + public void testAggregateWithSize() + { + Double[] values = new Double[]{3.0, 1.0, 2.0}; + TestObjectColumnSelector columnValueSelector = new TestObjectColumnSelector<>(values); + + ColumnSelectorFactory colSelectorFactory = EasyMock.mock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector(EasyMock.anyString())) + .andReturn(columnValueSelector).anyTimes(); + EasyMock.replay(colSelectorFactory); + + DoubleMeanAggregatorFactory aggregatorFactory = new DoubleMeanAggregatorFactory("name", "fieldName"); + AggregatorAndSize aggregatorAndSize = aggregatorFactory.factorizeWithSize(colSelectorFactory); + + Assert.assertEquals( + aggregatorFactory.getMaxIntermediateSize(), + aggregatorAndSize.getInitialSizeBytes() + ); + Assert.assertTrue(aggregatorAndSize.getAggregator() instanceof DoubleMeanAggregator); + Aggregator aggregator = aggregatorAndSize.getAggregator(); + for (int i = 0; i < values.length; ++i) { + long sizeDelta = aggregator.aggregateWithSize(); + Assert.assertEquals(0L, sizeDelta); + columnValueSelector.increment(); + } + + DoubleMeanHolder meanHolder = (DoubleMeanHolder) aggregator.get(); + Assert.assertEquals(2.0, meanHolder.mean(), 0.0); + } } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactoryTest.java index 01dabaaca7fa..faa445bf32dc 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactoryTest.java @@ -19,6 +19,10 @@ package org.apache.druid.query.aggregation.mean; +import org.apache.druid.query.aggregation.AggregatorAndSize; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -62,4 +66,18 @@ public void testFinalizeComputation() Assert.assertNull(factory.finalizeComputation(null)); } + + @Test + public void testFactorizeWithSize() + { + ColumnSelectorFactory colSelectorFactory = EasyMock.mock(ColumnSelectorFactory.class); + EasyMock.expect(colSelectorFactory.makeColumnValueSelector(EasyMock.anyString())) + .andReturn(EasyMock.createMock(ColumnValueSelector.class)).anyTimes(); + EasyMock.replay(colSelectorFactory); + + DoubleMeanAggregatorFactory factory = new DoubleMeanAggregatorFactory("name", "fieldName"); + AggregatorAndSize aggregatorAndSize = factory.factorizeWithSize(colSelectorFactory); + Assert.assertEquals(DoubleMeanHolder.MAX_INTERMEDIATE_SIZE, aggregatorAndSize.getInitialSizeBytes()); + Assert.assertTrue(aggregatorAndSize.getAggregator() instanceof DoubleMeanAggregator); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java index 949a8408d9ff..60540e2fe250 100644 --- a/processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/StringDimensionIndexerTest.java @@ -19,10 +19,9 @@ package org.apache.druid.segment; -import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import java.util.Arrays; @@ -30,14 +29,8 @@ /** * Unit tests for {@link StringDimensionIndexer}. */ -public class StringDimensionIndexerTest +public class StringDimensionIndexerTest extends InitializedNullHandlingTest { - @Before - public void setup() - { - NullHandling.initializeForTests(); - } - @Test public void testProcessRowValsToEncodedKeyComponent_usingAvgEstimates() { @@ -226,4 +219,4 @@ private long verifyEncodedValues( return encodedKeyComponent.getEffectiveSizeBytes(); } -} \ No newline at end of file +} 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 ea448fdf8edf..27ccf182e096 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 @@ -54,6 +54,7 @@ public IncrementalIndexRowSizeTest(String indexType) throws JsonProcessingExcept .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) .setMaxRowCount(10_000) .setMaxBytesInMemory(1_000) + .setUseMaxMemoryEstimates(true) .build()) ); } From 86a6e933fceb44af1bd62373ec42935142c052ec Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Thu, 20 Jan 2022 20:36:38 +0530 Subject: [PATCH 11/15] Fix propagation of context flag useMaxMemoryEstimates --- .../druid/segment/incremental/IncrementalIndex.java | 10 +++++++--- .../segment/incremental/OnheapIncrementalIndex.java | 12 +----------- 2 files changed, 8 insertions(+), 14 deletions(-) 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 5ffb65e465ed..86634b87e733 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 @@ -236,6 +236,7 @@ public ColumnCapabilities getColumnCapabilities(String columnName) private final Map timeAndMetricsColumnCapabilities; private final AtomicInteger numEntries = new AtomicInteger(); private final AtomicLong bytesInMemory = new AtomicLong(); + private final boolean useMaxMemoryEstimates; // This is modified on add() in a critical section. private final ThreadLocal in = new ThreadLocal<>(); @@ -255,11 +256,13 @@ public ColumnCapabilities getColumnCapabilities(String columnName) * @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input * value for aggregators that return metrics other than float. * @param concurrentEventAdd flag whether ot not adding of input rows should be thread-safe + * @param useMaxMemoryEstimates true if max values should be used to estimate memory */ protected IncrementalIndex( final IncrementalIndexSchema incrementalIndexSchema, final boolean deserializeComplexMetrics, - final boolean concurrentEventAdd + final boolean concurrentEventAdd, + final boolean useMaxMemoryEstimates ) { this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); @@ -269,6 +272,7 @@ protected IncrementalIndex( this.metrics = incrementalIndexSchema.getMetrics(); this.rowTransformers = new CopyOnWriteArrayList<>(); this.deserializeComplexMetrics = deserializeComplexMetrics; + this.useMaxMemoryEstimates = useMaxMemoryEstimates; this.timeAndMetricsColumnCapabilities = new HashMap<>(); this.metricDescs = Maps.newLinkedHashMap(); @@ -875,9 +879,9 @@ private DimensionDesc addNewDimension(String dim, DimensionHandler handler) return desc; } - protected DimensionDesc initDimension(int dimensionIndex, String dimensionName, DimensionHandler dimensionHandler) + private DimensionDesc initDimension(int dimensionIndex, String dimensionName, DimensionHandler dimensionHandler) { - return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler); + return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler, useMaxMemoryEstimates); } public List getMetricNames() 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 53a24f4b3bc4..493180db458e 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 @@ -121,7 +121,7 @@ public class OnheapIncrementalIndex extends IncrementalIndex boolean useMaxMemoryEstimates ) { - super(incrementalIndexSchema, deserializeComplexMetrics, concurrentEventAdd); + super(incrementalIndexSchema, deserializeComplexMetrics, concurrentEventAdd, useMaxMemoryEstimates); this.maxRowCount = maxRowCount; this.maxBytesInMemory = maxBytesInMemory == 0 ? Long.MAX_VALUE : maxBytesInMemory; this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) @@ -193,16 +193,6 @@ protected void initAggs( } } - @Override - protected DimensionDesc initDimension( - int dimensionIndex, - String dimensionName, - DimensionHandler dimensionHandler - ) - { - return new DimensionDesc(dimensionIndex, dimensionName, dimensionHandler, useMaxMemoryEstimates); - } - @Override protected AddToFactsResult addToFacts( InputRow row, From 000a9bf37c09a024decf58a9cfb454cb0c029f3f Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Fri, 21 Jan 2022 10:02:55 +0530 Subject: [PATCH 12/15] Remove unused DimensionDesc constructor --- .../apache/druid/segment/incremental/IncrementalIndex.java | 5 ----- 1 file changed, 5 deletions(-) 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 86634b87e733..484bc89fc955 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 @@ -940,11 +940,6 @@ public static final class DimensionDesc private final DimensionHandler handler; private final DimensionIndexer indexer; - public DimensionDesc(int index, String name, DimensionHandler handler) - { - this(index, name, handler, true); - } - public DimensionDesc(int index, String name, DimensionHandler handler, boolean useMaxMemoryEstimates) { this.index = index; From e5ca2ebff1794ec0a87f56f0400a5b6dca4753e7 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 1 Feb 2022 12:19:10 +0530 Subject: [PATCH 13/15] Initialize static fields in SketchAggregator through SketchModule --- .../datasketches/theta/SketchAggregator.java | 12 ++++++++++-- .../datasketches/theta/SketchModule.java | 1 + .../druid/query/aggregation/Aggregator.java | 4 ++++ .../query/aggregation/AggregatorFactory.java | 5 ++++- .../segment/StringDimensionDictionary.java | 18 ++++++++++++++---- .../druid/segment/StringDimensionIndexer.java | 2 +- 6 files changed, 34 insertions(+), 8 deletions(-) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java index de582992b1e7..fe7c9c52072c 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -45,9 +45,17 @@ public class SketchAggregator implements Aggregator private Sketch sketch; @Nullable - private static final Field SKETCH_FIELD; + private static Field SKETCH_FIELD; - static { + /** + * Initializes static fields of the SketchAggregator needed for memory + * estimation. + */ + public static void initialize() + { + if (SKETCH_FIELD != null) { + return; + } try { SKETCH_FIELD = Class.forName("org.apache.datasketches.theta.UnionImpl") .getDeclaredField("gadget_"); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java index 9d5746533ff6..3783d9e346a7 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchModule.java @@ -71,6 +71,7 @@ public void configure(Binder binder) ThetaSketchApproxCountDistinctSqlAggregator.NAME, ThetaSketchApproxCountDistinctSqlAggregator.class ); + SketchAggregator.initialize(); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java index e59a93cbf34d..a895d68d41a1 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/Aggregator.java @@ -50,6 +50,10 @@ public interface Aggregator extends Closeable * caused by this aggregation step. *

      * The default implementation of this method calls {@link #aggregate()} and returns 0. + *

      + * If overridden, this method must include the JVM object overheads in the size + * estimation and must ensure not to underestimate required memory as that might + * lead to OOM errors. * * @return Increase in required on-heap memory caused by this aggregation step. */ diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 59c850ba810c..9056423832d6 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -76,7 +76,10 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFact * {@link Aggregator#aggregateWithSize()} to perform aggregation and get back * the incremental memory required in each aggregate call. Combined with the * initial size, this gives the total on-heap memory required by the aggregator. - * + *

      + * This method must include JVM object overheads in the estimated size and must + * ensure not to underestimate required memory as that might lead to OOM errors. + *

      * This flow does not require invoking {@link #guessAggregatorHeapFootprint(long)} * which tends to over-estimate the required memory. * diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java index 3072f6d123c1..282924019d10 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionDictionary.java @@ -21,12 +21,22 @@ /** * DimensionDictionary for String dimension values. - *

      - * With StringDimensionDictionary, on-heap memory estimation of the dictionary - * size is always enabled. */ public class StringDimensionDictionary extends DimensionDictionary { + private final boolean computeOnHeapSize; + + /** + * Creates a StringDimensionDictionary. + * + * @param computeOnHeapSize true if on-heap memory estimation of the dictionary + * size should be enabled, false otherwise + */ + public StringDimensionDictionary(boolean computeOnHeapSize) + { + this.computeOnHeapSize = computeOnHeapSize; + } + @Override public long estimateSizeOfValue(String value) { @@ -38,6 +48,6 @@ public long estimateSizeOfValue(String value) @Override public boolean computeOnHeapSize() { - return true; + return computeOnHeapSize; } } diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index 7837253b93c6..c88f22f477be 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -70,7 +70,7 @@ public StringDimensionIndexer( boolean useMaxMemoryEstimates ) { - super(useMaxMemoryEstimates ? new DimensionDictionary<>() : new StringDimensionDictionary()); + super(new StringDimensionDictionary(!useMaxMemoryEstimates)); this.multiValueHandling = multiValueHandling == null ? MultiValueHandling.ofDefault() : multiValueHandling; this.hasBitmapIndexes = hasBitmapIndexes; this.hasSpatialIndexes = hasSpatialIndexes; From 6be53be80c2ff518ffa86d0ee3427aadf6b7ddf6 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 2 Feb 2022 12:15:18 +0530 Subject: [PATCH 14/15] Fix spotbug failure --- .../query/aggregation/datasketches/theta/SketchAggregator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java index fe7c9c52072c..dbf5c9dc262d 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -51,7 +51,7 @@ public class SketchAggregator implements Aggregator * Initializes static fields of the SketchAggregator needed for memory * estimation. */ - public static void initialize() + public static synchronized void initialize() { if (SKETCH_FIELD != null) { return; From 614a58bd5ca71cad9defa66d1adc9fc820ca49f9 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Wed, 2 Feb 2022 13:47:35 +0530 Subject: [PATCH 15/15] Fix SketchAggregationTest --- .../aggregation/datasketches/theta/SketchAggregationTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index fb36a0797657..6f0aaeb22699 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -549,6 +549,8 @@ public void testUpdateUnionWithDouble() @Test public void testAggregateWithSize() { + SketchAggregator.initialize(); + final String[] columnValues = new String[20]; for (int i = 0; i < columnValues.length; ++i) { columnValues[i] = "" + i;