diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java index 443c884b596d..f132962ee798 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java @@ -57,8 +57,13 @@ import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; import java.io.IOException; import java.util.ArrayList; @@ -73,7 +78,7 @@ @Measurement(iterations = 25) public class IncrementalIndexReadBenchmark { - @Param({"750000"}) + @Param({"7500"}) private int rowsPerSegment; @Param({"basic"}) @@ -82,6 +87,9 @@ public class IncrementalIndexReadBenchmark @Param({"true", "false"}) private boolean rollup; + @Param({"oak"}) + private String indexType; + private static final Logger log = new Logger(IncrementalIndexReadBenchmark.class); private static final int RNG_SEED = 9999; private IncrementalIndex incIndex; @@ -100,10 +108,10 @@ public void setup() throws IOException schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); BenchmarkDataGenerator gen = new BenchmarkDataGenerator( - schemaInfo.getColumnSchemas(), - RNG_SEED, - schemaInfo.getDataInterval(), - rowsPerSegment + schemaInfo.getColumnSchemas(), + RNG_SEED, + schemaInfo.getDataInterval(), + rowsPerSegment ); incIndex = makeIncIndex(); @@ -115,21 +123,42 @@ public void setup() throws IOException } incIndex.add(row); } + } + @TearDown + public void tearDown() + { + incIndex.close(); } private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMetrics(schemaInfo.getAggsArray()) - .withRollup(rollup) - .build() - ) - .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + switch (indexType) { + case "onheap": + return new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment) + .buildOnheap(); + case "oak": + return new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment) + .buildOffheapOak(); + + } + return null; } @Benchmark @@ -195,12 +224,12 @@ public void readWithFilters(Blackhole blackhole) private Sequence makeCursors(IncrementalIndexStorageAdapter sa, DimFilter filter) { return sa.makeCursors( - filter.toFilter(), - schemaInfo.getDataInterval(), - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null + (filter == null) ? null : filter.toFilter(), + schemaInfo.getDataInterval(), + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null ); } @@ -208,4 +237,16 @@ private static DimensionSelector makeDimensionSelector(Cursor cursor, String nam { return cursor.getColumnSelectorFactory().makeDimensionSelector(new DefaultDimensionSpec(name, null)); } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(IncrementalIndexReadBenchmark.class.getSimpleName()) + .forks(0) + .build(); + + new Runner(opt).run(); + } + + } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java index 76a0fd953f4d..f496764c4653 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java @@ -40,8 +40,13 @@ import org.openjdk.jmh.annotations.Scope; import org.openjdk.jmh.annotations.Setup; import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; import java.util.ArrayList; import java.util.concurrent.TimeUnit; @@ -61,6 +66,9 @@ public class IndexIngestionBenchmark @Param({"true", "false"}) private boolean rollup; + @Param({"onheap", "oak"}) + private String indexType; + private static final Logger log = new Logger(IndexIngestionBenchmark.class); private static final int RNG_SEED = 9999; @@ -98,18 +106,40 @@ public void setup2() incIndex = makeIncIndex(); } + @TearDown(Level.Invocation) + public void tearDown() + { + incIndex.close(); + } + private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMetrics(schemaInfo.getAggsArray()) - .withRollup(rollup) - .build() - ) - .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment * 2) - .buildOnheap(); + switch (indexType) { + case "onheap": + return new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment * 2) + .buildOnheap(); + case "oak": + return new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment * 2) + .buildOffheapOak(); + + } + return null; } @Benchmark @@ -123,4 +153,18 @@ public void addRows(Blackhole blackhole) throws Exception blackhole.consume(rv); } } + + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(IndexIngestionBenchmark.class.getSimpleName()) + .threads(1) + .forks(0) + .build(); + + new Runner(opt).run(); + } + + } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionMultiThreadedBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionMultiThreadedBenchmark.java new file mode 100644 index 000000000000..7d1050940743 --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionMultiThreadedBenchmark.java @@ -0,0 +1,191 @@ +/* + * 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.benchmark.indexing; + +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Param; +import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; +import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; +import org.apache.druid.benchmark.datagen.BenchmarkSchemas; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.hll.HyperLogLogHash; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.serde.ComplexMetrics; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.BenchmarkParams; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.util.ArrayList; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 25) +@Threads(8) +public class IndexIngestionMultiThreadedBenchmark +{ + @Param({"100"}) + private int rowsPerSegmentPerThread; + + @Param({"basic"}) + private String schema; + + @Param({"true", "false"}) + private boolean rollup; + + @Param({"oak", "onheap"}) + private String indexType; + + private static final Logger log = new Logger(IndexIngestionMultiThreadedBenchmark.class); + private static final int RNG_SEED = 9999; + + private IncrementalIndex incIndex; + private ArrayList rows; + private BenchmarkSchemaInfo schemaInfo; + private AtomicInteger startIndexdistributer = new AtomicInteger(0); + + @Setup + public void setup(BenchmarkParams params) + { + int threads = params.getThreads(); + ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(HyperLogLogHash.getDefault())); + + rows = new ArrayList(); + schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); + + BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + schemaInfo.getColumnSchemas(), + RNG_SEED, + schemaInfo.getDataInterval(), + rowsPerSegmentPerThread * threads + ); + + for (int i = 0; i < rowsPerSegmentPerThread * threads; i++) { + InputRow row = gen.nextRow(); + if (i % 10000 == 0) { + log.info(i + " rows generated."); + } + rows.add(row); + } + } + + @TearDown(Level.Iteration) + public void tearDown() + { + incIndex.close(); + } + + @State(Scope.Thread) + public static class ThreadState + { + int startIndex; + + @Setup + public void setup(IndexIngestionMultiThreadedBenchmark state) + { + startIndex = state.startIndexdistributer.getAndIncrement(); + } + } + + + @Setup(Level.Iteration) + public void setup2(BenchmarkParams params) + { + incIndex = makeIncIndex(params.getThreads() * rowsPerSegmentPerThread * 2); + } + + private IncrementalIndex makeIncIndex(int maxRowCount) + { + switch (indexType) { + case "oak": + { + return new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(maxRowCount) + .buildOffheapOak(); + } + case "onheap": { + return new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(maxRowCount) + .buildOnheap(); + } + default: + return null; + } + } + + @Benchmark + @BenchmarkMode(Mode.SingleShotTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void addRows(Blackhole blackhole, ThreadState threadState, BenchmarkParams params) throws Exception + { + for (int i = threadState.startIndex; i < rowsPerSegmentPerThread; i += params.getThreads()) { + InputRow row = rows.get(i); + int rv = incIndex.add(row).getRowCount(); + blackhole.consume(rv); + } + } + + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(IndexIngestionMultiThreadedBenchmark.class.getSimpleName()) + .threads(1) + .forks(1) + .build(); + + new Runner(opt).run(); + } + +} diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java index a1cddc735a66..771fda935625 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -83,6 +83,10 @@ import org.openjdk.jmh.annotations.TearDown; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; import java.io.File; import java.io.IOException; @@ -112,6 +116,9 @@ public class TopNBenchmark @Param({"10"}) private int threshold; + @Param({"onheap", "oak"}) + private String indexType; + private static final Logger log = new Logger(TopNBenchmark.class); private static final int RNG_SEED = 9999; private static final IndexMergerV9 INDEX_MERGER_V9; @@ -287,15 +294,26 @@ public void setup() throws IOException public void tearDown() throws IOException { FileUtils.deleteDirectory(tmpDir); + incIndexes.forEach(index -> index.close()); } private IncrementalIndex makeIncIndex() { - return new IncrementalIndex.Builder() - .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) - .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + switch (indexType) { + case "onheap": + return new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment) + .buildOnheap(); + case "oak": + return new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment) + .buildOffheapOak(); + } + return null; } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) @@ -379,4 +397,15 @@ public void queryMultiQueryableIndex(Blackhole blackhole) blackhole.consume(result); } } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(TopNBenchmark.class.getSimpleName()) + .threads(1) + .build(); + + new Runner(opt).run(); + } + } diff --git a/codestyle/checkstyle.xml b/codestyle/checkstyle.xml deleted file mode 100644 index eebbc5888dc8..000000000000 --- a/codestyle/checkstyle.xml +++ /dev/null @@ -1,202 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java index e0d01d08ec48..ede237f72760 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/InputRowSerde.java @@ -40,7 +40,7 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.AggsManager; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.hadoop.io.WritableUtils; @@ -312,7 +312,12 @@ public static final SerializeResult toBytes( writeString(k, out); try (Aggregator agg = aggFactory.factorize( - IncrementalIndex.makeColumnSelectorFactory(VirtualColumns.EMPTY, aggFactory, supplier, true) + AggsManager.makeColumnSelectorFactory( + VirtualColumns.EMPTY, + aggFactory, + supplier, + true + ) )) { try { agg.aggregate(); diff --git a/processing/pom.xml b/processing/pom.xml index ccb60ca9ef26..a1810a19f238 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -32,6 +32,11 @@ + + com.yahoo.oak + oak + 0.1.4-SNAPSHOT + org.apache.druid druid-common @@ -200,7 +205,7 @@ -Xmx512m - -XX:MaxDirectMemorySize=1500m + -XX:MaxDirectMemorySize=4g -Duser.language=en -Duser.GroupByQueryRunnerTest.javacountry=US -Dfile.encoding=UTF-8 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 b0159128c547..cbb0704fd119 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java @@ -27,6 +27,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.incremental.IncrementalIndexRow; import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; import javax.annotation.Nullable; @@ -106,20 +107,20 @@ class IndexerDoubleColumnSelector implements DoubleColumnSelector @Override public boolean isNull() { - final Object[] dims = currEntry.get().getDims(); - return dimIndex >= dims.length || dims[dimIndex] == null; + return currEntry.get().getDim(dimIndex) == null; } @Override public double getDouble() { - final Object[] dims = currEntry.get().getDims(); + IncrementalIndexRow incrementalIndexRow = currEntry.get(); + Object dim = incrementalIndexRow.getDim(dimIndex); - if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (dim == null) { assert NullHandling.replaceWithDefault(); return 0.0; } - return (Double) dims[dimIndex]; + return (Double) dim; } @SuppressWarnings("deprecation") @@ -127,12 +128,14 @@ public double getDouble() @Override public Double getObject() { - final Object[] dims = currEntry.get().getDims(); + IncrementalIndexRow incrementalIndexRow = currEntry.get(); + Object dim = incrementalIndexRow.getDim(dimIndex); - if (dimIndex >= dims.length) { - return null; + if (dim == null) { + assert NullHandling.replaceWithDefault(); + return 0.0; } - return (Double) dims[dimIndex]; + return (Double) dim; } @Override 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 ea92839d764c..f57950971a73 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java @@ -107,21 +107,20 @@ class IndexerFloatColumnSelector implements FloatColumnSelector @Override public boolean isNull() { - final Object[] dims = currEntry.get().getDims(); - return dimIndex >= dims.length || dims[dimIndex] == null; + return currEntry.get().getDim(dimIndex) == null; } @Override public float getFloat() { - final Object[] dims = currEntry.get().getDims(); + final Object dim = currEntry.get().getDim(dimIndex); - if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (dim == null) { assert NullHandling.replaceWithDefault(); return 0.0f; } - return (Float) dims[dimIndex]; + return (Float) dim; } @SuppressWarnings("deprecation") @@ -129,13 +128,8 @@ public float getFloat() @Override public Float getObject() { - final Object[] dims = currEntry.get().getDims(); - - if (dimIndex >= dims.length) { - return null; - } - - return (Float) dims[dimIndex]; + final Object dim = currEntry.get().getDim(dimIndex); + return dim == null ? null : (Float) dim; } @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 0f7c13d24ac6..a742670096b0 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java @@ -107,21 +107,20 @@ class IndexerLongColumnSelector implements LongColumnSelector @Override public boolean isNull() { - final Object[] dims = currEntry.get().getDims(); - return dimIndex >= dims.length || dims[dimIndex] == null; + return currEntry.get().getDim(dimIndex) == null; } @Override public long getLong() { - final Object[] dims = currEntry.get().getDims(); + final Object dim = currEntry.get().getDim(dimIndex); - if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (dim == null) { assert NullHandling.replaceWithDefault(); return 0; } - return (Long) dims[dimIndex]; + return (Long) dim; } @SuppressWarnings("deprecation") @@ -129,13 +128,8 @@ public long getLong() @Override public Long getObject() { - final Object[] dims = currEntry.get().getDims(); - - if (dimIndex >= dims.length) { - return null; - } - - return (Long) dims[dimIndex]; + final Object dim = currEntry.get().getDim(dimIndex); + return dim == null ? null : (Long) dim; } @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 19aef7344a1a..4af05bbe3407 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -442,20 +442,13 @@ class IndexerDimensionSelector implements DimensionSelector, IdLookup @Override public IndexedInts getRow() { - final Object[] dims = currEntry.get().getDims(); - - int[] indices; - if (dimIndex < dims.length) { - indices = (int[]) dims[dimIndex]; - } else { - indices = null; - } - - int[] row = null; - int rowSize = 0; + IncrementalIndexRow key = currEntry.get(); // usually due to currEntry's rowIndex is smaller than the row's rowIndex in which this dim first appears - if (indices == null || indices.length == 0) { + if (key.getDimsLength() == 0 || key.calcStringDimSize(dimIndex) == 0) { + int[] row; + int rowSize; + if (hasMultipleValues) { row = IntArrays.EMPTY_ARRAY; rowSize = 0; @@ -474,14 +467,11 @@ public IndexedInts getRow() rowSize = 0; } } + indexedInts.setValues(row, rowSize); + } else { + indexedInts.setValues(key, dimIndex); } - if (row == null && indices != null && indices.length > 0) { - row = indices; - rowSize = indices.length; - } - - indexedInts.setValues(row, rowSize); return indexedInts; } @@ -496,12 +486,12 @@ public ValueMatcher makeValueMatcher(final String value) @Override public boolean matches() { - Object[] dims = currEntry.get().getDims(); - if (dimIndex >= dims.length) { + IncrementalIndexRow key = currEntry.get(); + if (dimIndex >= key.getDimsLength()) { return value == null; } - int[] dimsInt = (int[]) dims[dimIndex]; + int[] dimsInt = (int[]) key.getDim(dimIndex); if (dimsInt == null || dimsInt.length == 0) { return value == null; } @@ -542,12 +532,12 @@ public ValueMatcher makeValueMatcher(final Predicate predicate) @Override public boolean matches() { - Object[] dims = currEntry.get().getDims(); - if (dimIndex >= dims.length) { + IncrementalIndexRow key = currEntry.get(); + if (dimIndex >= key.getDimsLength()) { return matchNull; } - int[] dimsInt = (int[]) dims[dimIndex]; + int[] dimsInt = (int[]) key.getDim(dimIndex); if (dimsInt == null || dimsInt.length == 0) { return matchNull; } @@ -627,12 +617,11 @@ public Object getObject() return null; } - Object[] dims = key.getDims(); - if (dimIndex >= dims.length) { + if (dimIndex >= key.getDimsLength()) { return null; } - return convertUnsortedEncodedKeyComponentToActualList((int[]) dims[dimIndex]); + return convertUnsortedEncodedKeyComponentToActualList((int[]) key.getDim(dimIndex)); } @SuppressWarnings("deprecation") diff --git a/processing/src/main/java/org/apache/druid/segment/data/ArrayBasedIndexedInts.java b/processing/src/main/java/org/apache/druid/segment/data/ArrayBasedIndexedInts.java index 2c48674c46bf..945d93ad0e85 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/ArrayBasedIndexedInts.java +++ b/processing/src/main/java/org/apache/druid/segment/data/ArrayBasedIndexedInts.java @@ -22,6 +22,8 @@ import it.unimi.dsi.fastutil.ints.IntArrays; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.incremental.IncrementalIndexRow; + /** */ @@ -73,6 +75,12 @@ public void setValues(int[] values, int size) this.size = size; } + public void setValues(IncrementalIndexRow row, int dimIndex) + { + ensureSize(row.calcStringDimSize(dimIndex)); + this.size = row.copyStringDim(dimIndex, expansion); + } + public void setValue(int index, int value) { expansion[index] = value; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/AggsManager.java b/processing/src/main/java/org/apache/druid/segment/incremental/AggsManager.java new file mode 100644 index 000000000000..f03330ef960a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/AggsManager.java @@ -0,0 +1,454 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.common.guava.GuavaUtils; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.ObjectColumnSelector; +import org.apache.druid.segment.LongColumnSelector; +import org.apache.druid.segment.FloatColumnSelector; +import org.apache.druid.segment.DoubleColumnSelector; +import org.apache.druid.segment.NilColumnValueSelector; +import org.apache.druid.segment.serde.ComplexMetricExtractor; +import org.apache.druid.segment.serde.ComplexMetricSerde; +import org.apache.druid.segment.serde.ComplexMetrics; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.List; +import java.util.concurrent.locks.ReentrantLock; + +public abstract class AggsManager +{ + + protected final AggregatorFactory[] metrics; + protected final AggregatorType[] aggs; + protected final boolean deserializeComplexMetrics; + private final Map columnCapabilities; + private final VirtualColumns virtualColumns; + + protected final Map metricDescs; + public IncrementalIndex incrementalIndex; + + public final ReentrantLock[] aggLocks; + + public AggsManager( + final IncrementalIndexSchema incrementalIndexSchema, + final boolean deserializeComplexMetrics, + final boolean concurrentEventAdd, + Supplier rowSupplier, + Map columnCapabilities, + IncrementalIndex incrementalIndex + ) + { + this.incrementalIndex = incrementalIndex; + this.virtualColumns = incrementalIndexSchema.getVirtualColumns(); + this.metrics = incrementalIndexSchema.getMetrics(); + this.deserializeComplexMetrics = deserializeComplexMetrics; + this.columnCapabilities = columnCapabilities; + this.aggs = initAggs(metrics, rowSupplier, deserializeComplexMetrics, concurrentEventAdd); + this.aggLocks = new ReentrantLock[this.aggs.length]; + for (int i = 0; i < this.aggLocks.length; i++) { + this.aggLocks[i] = new ReentrantLock(true); + } + this.metricDescs = Maps.newLinkedHashMap(); + for (AggregatorFactory metric : metrics) { + MetricDesc metricDesc = new MetricDesc(metricDescs.size(), metric); + metricDescs.put(metricDesc.getName(), metricDesc); + this.columnCapabilities.put(metricDesc.getName(), metricDesc.getCapabilities()); + } + } + + protected abstract AggregatorType[] initAggs( + AggregatorFactory[] metrics, + Supplier rowSupplier, + boolean deserializeComplexMetrics, + boolean concurrentEventAdd + ); + + public AggregatorType[] getAggs() + { + return aggs; + } + + public AggregatorFactory[] getMetricAggs() + { + return metrics; + } + + @Nullable + public String getMetricType(String metric) + { + final MetricDesc metricDesc = metricDescs.get(metric); + return metricDesc != null ? metricDesc.getType() : null; + } + + public List getMetricNames() + { + return ImmutableList.copyOf(metricDescs.keySet()); + } + + public static AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] aggregators) + { + AggregatorFactory[] combiningAggregators = new AggregatorFactory[aggregators.length]; + for (int i = 0; i < aggregators.length; i++) { + combiningAggregators[i] = aggregators[i].getCombiningFactory(); + } + return combiningAggregators; + } + + protected ColumnSelectorFactory makeColumnSelectorFactory( + final AggregatorFactory agg, + final Supplier in, + final boolean deserializeComplexMetrics + ) + { + return makeColumnSelectorFactory(virtualColumns, agg, in, deserializeComplexMetrics); + } + + /** + * Column selector used at ingestion time for inputs to aggregators. + * + * @param agg the aggregator + * @param in ingestion-time input row supplier + * @param deserializeComplexMetrics whether complex objects should be deserialized by a {@link ComplexMetricExtractor} + * + * @return column selector factory + */ + public static ColumnSelectorFactory makeColumnSelectorFactory( + final VirtualColumns virtualColumns, + final AggregatorFactory agg, + final Supplier in, + final boolean deserializeComplexMetrics + ) + { + final RowBasedColumnSelectorFactory baseSelectorFactory = RowBasedColumnSelectorFactory.create(in, null); + + class IncrementalIndexInputRowColumnSelectorFactory implements ColumnSelectorFactory + { + @Override + public ColumnValueSelector makeColumnValueSelector(final String column) + { + final String typeName = agg.getTypeName(); + boolean isComplexMetric = + GuavaUtils.getEnumIfPresent(ValueType.class, StringUtils.toUpperCase(typeName)) == null || + typeName.equalsIgnoreCase(ValueType.COMPLEX.name()); + if (!isComplexMetric || !deserializeComplexMetrics) { + return baseSelectorFactory.makeColumnValueSelector(column); + } else { + final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); + if (serde == null) { + throw new ISE("Don't know how to handle type[%s]", typeName); + } + + final ComplexMetricExtractor extractor = serde.getExtractor(); + return new ColumnValueSelector() + { + @Override + public boolean isNull() + { + return in.get().getMetric(column) == null; + } + + @Override + public long getLong() + { + Number metric = in.get().getMetric(column); + assert NullHandling.replaceWithDefault() || metric != null; + return DimensionHandlerUtils.nullToZero(metric).longValue(); + } + + @Override + public float getFloat() + { + Number metric = in.get().getMetric(column); + assert NullHandling.replaceWithDefault() || metric != null; + return DimensionHandlerUtils.nullToZero(metric).floatValue(); + } + + @Override + public double getDouble() + { + Number metric = in.get().getMetric(column); + assert NullHandling.replaceWithDefault() || metric != null; + return DimensionHandlerUtils.nullToZero(metric).doubleValue(); + } + + @Override + public Class classOfObject() + { + return extractor.extractedClass(); + } + + @Override + public Object getObject() + { + return extractor.extractValue(in.get(), column); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("in", in); + inspector.visit("extractor", extractor); + } + }; + } + } + + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + return baseSelectorFactory.makeDimensionSelector(dimensionSpec); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String columnName) + { + return baseSelectorFactory.getColumnCapabilities(columnName); + } + } + + return virtualColumns.wrap(new IncrementalIndexInputRowColumnSelectorFactory()); + } + + private class LongMetricColumnSelector implements LongColumnSelector + { + private final IncrementalIndexRowHolder currEntry; + private final int metricIndex; + + public LongMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + { + this.currEntry = currEntry; + this.metricIndex = metricIndex; + } + + @Override + public long getLong() + { + assert NullHandling.replaceWithDefault() || !isNull(); + return AggsManager.this.incrementalIndex.getMetricLongValue(currEntry.get(), metricIndex); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("index", AggsManager.this.incrementalIndex); + } + + @Override + public boolean isNull() + { + return AggsManager.this.incrementalIndex.isNull(currEntry.get(), metricIndex); + } + } + + private class ObjectMetricColumnSelector extends ObjectColumnSelector + { + private final IncrementalIndexRowHolder currEntry; + private final int metricIndex; + private Class classOfObject; + + public ObjectMetricColumnSelector( + MetricDesc metricDesc, + IncrementalIndexRowHolder currEntry, + int metricIndex + ) + { + this.currEntry = currEntry; + this.metricIndex = metricIndex; + classOfObject = ComplexMetrics.getSerdeForType(metricDesc.getType()).getObjectStrategy().getClazz(); + } + + @Nullable + @Override + public Object getObject() + { + return AggsManager.this.incrementalIndex.getMetricObjectValue(currEntry.get(), metricIndex); + } + + @Override + public Class classOfObject() + { + return classOfObject; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("index", AggsManager.this.incrementalIndex); + } + } + + private class FloatMetricColumnSelector implements FloatColumnSelector + { + private final IncrementalIndexRowHolder currEntry; + private final int metricIndex; + + public FloatMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + { + this.currEntry = currEntry; + this.metricIndex = metricIndex; + } + + @Override + public float getFloat() + { + assert NullHandling.replaceWithDefault() || !isNull(); + return AggsManager.this.incrementalIndex.getMetricFloatValue(currEntry.get(), metricIndex); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("index", AggsManager.this.incrementalIndex); + } + + @Override + public boolean isNull() + { + return AggsManager.this.incrementalIndex.isNull(currEntry.get(), metricIndex); + } + } + + private class DoubleMetricColumnSelector implements DoubleColumnSelector + { + private final IncrementalIndexRowHolder currEntry; + private final int metricIndex; + + public DoubleMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) + { + this.currEntry = currEntry; + this.metricIndex = metricIndex; + } + + @Override + public double getDouble() + { + assert NullHandling.replaceWithDefault() || !isNull(); + return AggsManager.this.incrementalIndex.getMetricDoubleValue(currEntry.get(), metricIndex); + } + + @Override + public boolean isNull() + { + return AggsManager.this.incrementalIndex.isNull(currEntry.get(), metricIndex); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("index", AggsManager.this.incrementalIndex); + } + } + + public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) + { + MetricDesc metricDesc = metricDescs.get(metric); + if (metricDesc == null) { + return NilColumnValueSelector.instance(); + } + int metricIndex = metricDesc.getIndex(); + switch (metricDesc.getCapabilities().getType()) { + case COMPLEX: + return new ObjectMetricColumnSelector(metricDesc, currEntry, metricIndex); + case LONG: + return new LongMetricColumnSelector(currEntry, metricIndex); + case FLOAT: + return new FloatMetricColumnSelector(currEntry, metricIndex); + case DOUBLE: + return new DoubleMetricColumnSelector(currEntry, metricIndex); + case STRING: + throw new IllegalStateException("String is not a metric column type"); + default: + throw new ISE("Unknown metric value type: %s", metricDesc.getCapabilities().getType()); + } + } + + public static final class MetricDesc + { + private final int index; + private final String name; + private final String type; + private final ColumnCapabilitiesImpl capabilities; + + public MetricDesc(int index, AggregatorFactory factory) + { + this.index = index; + this.name = factory.getName(); + + String typeInfo = factory.getTypeName(); + this.capabilities = new ColumnCapabilitiesImpl(); + if (typeInfo.equalsIgnoreCase("float")) { + capabilities.setType(ValueType.FLOAT); + this.type = typeInfo; + } else if (typeInfo.equalsIgnoreCase("long")) { + capabilities.setType(ValueType.LONG); + this.type = typeInfo; + } else if (typeInfo.equalsIgnoreCase("double")) { + capabilities.setType(ValueType.DOUBLE); + this.type = typeInfo; + } else { + capabilities.setType(ValueType.COMPLEX); + this.type = ComplexMetrics.getSerdeForType(typeInfo).getTypeName(); + } + } + + public int getIndex() + { + return index; + } + + public String getName() + { + return name; + } + + public String getType() + { + return type; + } + + public ColumnCapabilitiesImpl getCapabilities() + { + return capabilities; + } + } + +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/ExternalDataIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/ExternalDataIncrementalIndex.java new file mode 100644 index 000000000000..1b27f76f7013 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/ExternalDataIncrementalIndex.java @@ -0,0 +1,445 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import com.google.common.base.Supplier; +import com.google.common.collect.Iterators; +import com.google.common.collect.Maps; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.DimensionIndexer; + +import java.util.Map; +import java.util.Comparator; +import java.util.List; +import java.util.Iterator; +import java.util.Deque; +import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedDeque; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ConcurrentNavigableMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; + +/** + */ +public abstract class ExternalDataIncrementalIndex extends IncrementalIndex +{ + protected FactsHolder facts; + + /* basic constractor */ + ExternalDataIncrementalIndex( + IncrementalIndexSchema incrementalIndexSchema, + boolean reportParseExceptions, + boolean sortFacts, + int maxRowCount + ) + { + super(incrementalIndexSchema, reportParseExceptions, maxRowCount); + this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) + : new PlainFactsHolder(sortFacts, dimsComparator()); + } + + protected abstract FactsHolder getFacts(); + + // Note: This method needs to be thread safe. + protected abstract AddToFactsResult addToFacts( + boolean reportParseExceptions, + InputRow row, + AtomicInteger numEntries, + AtomicLong sizeInBytes, + IncrementalIndexRow key, + ThreadLocal rowContainer, + Supplier rowSupplier, + boolean skipMaxRowsInMemoryCheck + ) throws IndexSizeExceededException; + + @Override + public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException + { + IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row); + final AddToFactsResult addToFactsResult = addToFacts( + reportParseExceptions, + row, + numEntries, + bytesInMemory, + incrementalIndexRowResult.getIncrementalIndexRow(), + in, + rowSupplier, + skipMaxRowsInMemoryCheck + ); + updateMaxIngestedTime(row.getTimestamp()); + ParseException parseException = getCombinedParseException( + row, + incrementalIndexRowResult.getParseExceptionMessages(), + addToFactsResult.getParseExceptionMessages() + ); + return new IncrementalIndexAddResult(addToFactsResult.getRowCount(), addToFactsResult.getBytesInMemory(), parseException); + } + + @Override + protected long getMinTimeMillis() + { + return getFacts().getMinTimeMillis(); + } + + @Override + protected long getMaxTimeMillis() + { + return getFacts().getMaxTimeMillis(); + } + + protected abstract String getMetricName(int metricIndex); + + @Override + public Iterable timeRangeIterable( + boolean descending, long timeStart, long timeEnd) + { + return getFacts().timeRangeIterable(descending, timeStart, timeEnd); + } + + @Override + public Iterable keySet() + { + return getFacts().keySet(); + } + + @Override + public Iterable persistIterable() + { + return getFacts().persistIterable(); + } + + @Override + public Iterable iterableWithPostAggregations(final List postAggs, final boolean descending) + { + return new Iterable() + { + @Override + public Iterator iterator() + { + final List dimensions = getDimensions(); + + return Iterators.transform( + getFacts().iterator(descending), + incrementalIndexRow -> { + + Object[] theDims = incrementalIndexRow.getDims(); + + Map theVals = Maps.newLinkedHashMap(); + for (int i = 0; i < theDims.length; ++i) { + Object dim = theDims[i]; + DimensionDesc dimensionDesc = dimensions.get(i); + if (dimensionDesc == null) { + continue; + } + String dimensionName = dimensionDesc.getName(); + DimensionHandler handler = dimensionDesc.getHandler(); + if (dim == null || handler.getLengthOfEncodedKeyComponent(dim) == 0) { + theVals.put(dimensionName, null); + continue; + } + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + Object rowVals = indexer.convertUnsortedEncodedKeyComponentToActualList(dim); + theVals.put(dimensionName, rowVals); + } + + AggregatorType[] aggs = getAggsForRow(incrementalIndexRow); + for (int i = 0; i < aggs.length; ++i) { + theVals.put(getMetricName(i), getAggVal(incrementalIndexRow, i)); + } + + if (postAggs != null) { + for (PostAggregator postAgg : postAggs) { + theVals.put(postAgg.getName(), postAgg.compute(theVals)); + } + } + + return new MapBasedRow(incrementalIndexRow.getTimestamp(), theVals); + } + ); + } + }; + } + + + interface FactsHolder + { + /** + * @return the previous rowIndex associated with the specified key, or + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. + */ + int getPriorIndex(IncrementalIndexRow key); + + long getMinTimeMillis(); + + long getMaxTimeMillis(); + + Iterator iterator(boolean descending); + + Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); + + Iterable keySet(); + + /** + * Get all {@link IncrementalIndexRow} to persist, ordered with {@link Comparator} + * @return + */ + Iterable persistIterable(); + + /** + * @return the previous rowIndex associated with the specified key, or + * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. + */ + int putIfAbsent(IncrementalIndexRow key, int rowIndex); + + void clear(); + } + + static class RollupFactsHolder implements FactsHolder + { + private final boolean sortFacts; + // Can't use Set because we need to be able to get from collection + private final ConcurrentMap facts; + private final List dimensionDescsList; + + RollupFactsHolder( + boolean sortFacts, + Comparator incrementalIndexRowComparator, + List dimensionDescsList + ) + { + this.sortFacts = sortFacts; + if (sortFacts) { + this.facts = new ConcurrentSkipListMap<>(incrementalIndexRowComparator); + } else { + this.facts = new ConcurrentHashMap<>(); + } + this.dimensionDescsList = dimensionDescsList; + } + + @Override + public int getPriorIndex(IncrementalIndexRow key) + { + IncrementalIndexRow row = facts.get(key); + return row == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : row.getRowIndex(); + } + + @Override + public long getMinTimeMillis() + { + if (sortFacts) { + return ((ConcurrentNavigableMap) facts).firstKey().getTimestamp(); + } else { + throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); + } + } + + @Override + public long getMaxTimeMillis() + { + if (sortFacts) { + return ((ConcurrentNavigableMap) facts).lastKey().getTimestamp(); + } else { + throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); + } + } + + @Override + public Iterator iterator(boolean descending) + { + if (descending && sortFacts) { + return ((ConcurrentNavigableMap) facts).descendingMap().keySet().iterator(); + } + return keySet().iterator(); + } + + @Override + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) + { + if (!sortFacts) { + throw new UnsupportedOperationException("can't get timeRange from unsorted facts data."); + } + IncrementalIndexRow start = new IncrementalIndexRow(timeStart, new Object[]{}, dimensionDescsList); + IncrementalIndexRow end = new IncrementalIndexRow(timeEnd, new Object[]{}, dimensionDescsList); + ConcurrentNavigableMap subMap = + ((ConcurrentNavigableMap) facts).subMap(start, end); + final Map rangeMap = descending ? subMap.descendingMap() : subMap; + return rangeMap.keySet(); + } + + @Override + public Iterable keySet() + { + return facts.keySet(); + } + + @Override + public Iterable persistIterable() + { + // with rollup, facts are already pre-sorted so just return keyset + return keySet(); + } + + @Override + public int putIfAbsent(IncrementalIndexRow key, int rowIndex) + { + // setRowIndex() must be called before facts.putIfAbsent() for visibility of rowIndex from concurrent readers. + key.setRowIndex(rowIndex); + IncrementalIndexRow prev = facts.putIfAbsent(key, key); + return prev == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : prev.getRowIndex(); + } + + @Override + public void clear() + { + facts.clear(); + } + } + + static class PlainFactsHolder implements FactsHolder + { + private final boolean sortFacts; + private final ConcurrentMap> facts; + + private final Comparator incrementalIndexRowComparator; + + public PlainFactsHolder(boolean sortFacts, Comparator incrementalIndexRowComparator) + { + this.sortFacts = sortFacts; + if (sortFacts) { + this.facts = new ConcurrentSkipListMap<>(); + } else { + this.facts = new ConcurrentHashMap<>(); + } + this.incrementalIndexRowComparator = incrementalIndexRowComparator; + } + + @Override + public int getPriorIndex(IncrementalIndexRow key) + { + // always return EMPTY_ROW_INDEX to indicate that no prior key cause we always add new row + return IncrementalIndexRow.EMPTY_ROW_INDEX; + } + + @Override + public long getMinTimeMillis() + { + if (sortFacts) { + return ((ConcurrentNavigableMap>) facts).firstKey(); + } else { + throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); + } + } + + @Override + public long getMaxTimeMillis() + { + if (sortFacts) { + return ((ConcurrentNavigableMap>) facts).lastKey(); + } else { + throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); + } + } + + @Override + public Iterator iterator(boolean descending) + { + if (descending && sortFacts) { + return timeOrderedConcat(((ConcurrentNavigableMap>) facts) + .descendingMap().values(), true).iterator(); + } + return timeOrderedConcat(facts.values(), false).iterator(); + } + + @Override + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) + { + ConcurrentNavigableMap> subMap = + ((ConcurrentNavigableMap>) facts).subMap(timeStart, timeEnd); + final Map> rangeMap = descending ? subMap.descendingMap() : subMap; + return timeOrderedConcat(rangeMap.values(), descending); + } + + private Iterable timeOrderedConcat( + final Iterable> iterable, + final boolean descending + ) + { + return () -> Iterators.concat( + Iterators.transform(iterable.iterator(), + input -> descending ? input.descendingIterator() : input.iterator()) + ); + } + + private Stream timeAndDimsOrderedConcat( + final Collection> rowGroups + ) + { + return rowGroups.stream() + .flatMap(Collection::stream) + .sorted(incrementalIndexRowComparator); + } + + @Override + public Iterable keySet() + { + return timeOrderedConcat(facts.values(), false); + } + + @Override + public Iterable persistIterable() + { + return () -> timeAndDimsOrderedConcat(facts.values()).iterator(); + } + + @Override + public int putIfAbsent(IncrementalIndexRow key, int rowIndex) + { + Long time = key.getTimestamp(); + Deque rows = facts.get(time); + if (rows == null) { + facts.putIfAbsent(time, new ConcurrentLinkedDeque<>()); + // in race condition, rows may be put by other thread, so always get latest status from facts + rows = facts.get(time); + } + // setRowIndex() must be called before rows.add() for visibility of rowIndex from concurrent readers. + key.setRowIndex(rowIndex); + rows.add(key); + // always return EMPTY_ROW_INDEX to indicate that we always add new row + return IncrementalIndexRow.EMPTY_ROW_INDEX; + } + + @Override + public void clear() + { + facts.clear(); + } + } + +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index 08d722544db6..77c41b6274f2 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 @@ -25,16 +25,12 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import org.apache.druid.collections.NonBlockingPool; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.data.input.InputRow; -import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -42,36 +38,21 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; -import org.apache.druid.query.dimension.DimensionSpec; -import org.apache.druid.query.groupby.RowBasedColumnSelectorFactory; -import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractIndex; -import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionHandler; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionIndexer; -import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.DoubleColumnSelector; -import org.apache.druid.segment.FloatColumnSelector; -import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.Metadata; -import org.apache.druid.segment.NilColumnValueSelector; -import org.apache.druid.segment.ObjectColumnSelector; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.serde.ComplexMetricExtractor; -import org.apache.druid.segment.serde.ComplexMetricSerde; -import org.apache.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -80,172 +61,52 @@ import java.io.Closeable; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Collection; import java.util.Comparator; -import java.util.Deque; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedDeque; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ConcurrentNavigableMap; -import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Stream; /** */ public abstract class IncrementalIndex extends AbstractIndex implements Iterable, Closeable { - private volatile DateTime maxIngestedEventTime; + protected volatile DateTime maxIngestedEventTime; // Used to discover ValueType based on the class of values in a row // Also used to convert between the duplicate ValueType enums in DimensionSchema (druid-api) and main druid. public static final Map TYPE_MAP = ImmutableMap.builder() - .put(Long.class, ValueType.LONG) - .put(Double.class, ValueType.DOUBLE) - .put(Float.class, ValueType.FLOAT) - .put(String.class, ValueType.STRING) - .put(DimensionSchema.ValueType.LONG, ValueType.LONG) - .put(DimensionSchema.ValueType.FLOAT, ValueType.FLOAT) - .put(DimensionSchema.ValueType.STRING, ValueType.STRING) - .put(DimensionSchema.ValueType.DOUBLE, ValueType.DOUBLE) - .build(); - - /** - * Column selector used at ingestion time for inputs to aggregators. - * - * @param agg the aggregator - * @param in ingestion-time input row supplier - * @param deserializeComplexMetrics whether complex objects should be deserialized by a {@link ComplexMetricExtractor} - * - * @return column selector factory - */ - public static ColumnSelectorFactory makeColumnSelectorFactory( - final VirtualColumns virtualColumns, - final AggregatorFactory agg, - final Supplier in, - final boolean deserializeComplexMetrics - ) - { - final RowBasedColumnSelectorFactory baseSelectorFactory = RowBasedColumnSelectorFactory.create(in, null); - - class IncrementalIndexInputRowColumnSelectorFactory implements ColumnSelectorFactory - { - @Override - public ColumnValueSelector makeColumnValueSelector(final String column) - { - final String typeName = agg.getTypeName(); - boolean isComplexMetric = - GuavaUtils.getEnumIfPresent(ValueType.class, StringUtils.toUpperCase(typeName)) == null || - typeName.equalsIgnoreCase(ValueType.COMPLEX.name()); - if (!isComplexMetric || !deserializeComplexMetrics) { - return baseSelectorFactory.makeColumnValueSelector(column); - } else { - final ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(typeName); - if (serde == null) { - throw new ISE("Don't know how to handle type[%s]", typeName); - } - - final ComplexMetricExtractor extractor = serde.getExtractor(); - return new ColumnValueSelector() - { - @Override - public boolean isNull() - { - return in.get().getMetric(column) == null; - } - - @Override - public long getLong() - { - Number metric = in.get().getMetric(column); - assert NullHandling.replaceWithDefault() || metric != null; - return DimensionHandlerUtils.nullToZero(metric).longValue(); - } - - @Override - public float getFloat() - { - Number metric = in.get().getMetric(column); - assert NullHandling.replaceWithDefault() || metric != null; - return DimensionHandlerUtils.nullToZero(metric).floatValue(); - } - - @Override - public double getDouble() - { - Number metric = in.get().getMetric(column); - assert NullHandling.replaceWithDefault() || metric != null; - return DimensionHandlerUtils.nullToZero(metric).doubleValue(); - } - - @Override - public Class classOfObject() - { - return extractor.extractedClass(); - } - - @Nullable - @Override - public Object getObject() - { - return extractor.extractValue(in.get(), column); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("in", in); - inspector.visit("extractor", extractor); - } - }; - } - } - - @Override - public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) - { - return baseSelectorFactory.makeDimensionSelector(dimensionSpec); - } - - @Nullable - @Override - public ColumnCapabilities getColumnCapabilities(String columnName) - { - return baseSelectorFactory.getColumnCapabilities(columnName); - } - } - - return virtualColumns.wrap(new IncrementalIndexInputRowColumnSelectorFactory()); - } - - private final long minTimestamp; + .put(Long.class, ValueType.LONG) + .put(Double.class, ValueType.DOUBLE) + .put(Float.class, ValueType.FLOAT) + .put(String.class, ValueType.STRING) + .put(DimensionSchema.ValueType.LONG, ValueType.LONG) + .put(DimensionSchema.ValueType.FLOAT, ValueType.FLOAT) + .put(DimensionSchema.ValueType.STRING, ValueType.STRING) + .put(DimensionSchema.ValueType.DOUBLE, ValueType.DOUBLE) + .build(); + + protected final long minTimestamp; private final Granularity gran; private final boolean rollup; private final List> rowTransformers; - private final VirtualColumns virtualColumns; - private final AggregatorFactory[] metrics; - private final AggregatorType[] aggs; - private final boolean deserializeComplexMetrics; - private final boolean reportParseExceptions; // only used by OffHeapIncrementalIndex + protected final boolean reportParseExceptions; private final Metadata metadata; - private final Map metricDescs; - private final Map dimensionDescs; - private final List dimensionDescsList; - private final Map columnCapabilities; - private final AtomicInteger numEntries = new AtomicInteger(); - private final AtomicLong bytesInMemory = new AtomicLong(); + protected final List dimensionDescsList; + protected final Map columnCapabilities; + protected final AtomicInteger numEntries = new AtomicInteger(); + protected final AtomicLong bytesInMemory = new AtomicLong(); + protected final int maxRowCount; + protected String outOfRowsReason = null; // This is modified on add() in a critical section. - private final ThreadLocal in = new ThreadLocal<>(); - private final Supplier rowSupplier = in::get; + protected final ThreadLocal in = new ThreadLocal<>(); + protected final Supplier rowSupplier = in::get; /** * Setting deserializeComplexMetrics to false is necessary for intermediate aggregation such as groupBy that @@ -255,46 +116,32 @@ public ColumnCapabilities getColumnCapabilities(String columnName) * where the multiple threads can add concurrently to the IncrementalIndex). * * @param incrementalIndexSchema the schema to use for incremental index - * @param deserializeComplexMetrics flag whether or not to call ComplexMetricExtractor.extractValue() on the input - * value for aggregators that return metrics other than float. * @param reportParseExceptions flag whether or not to report ParseExceptions that occur while extracting values * from input rows - * @param concurrentEventAdd flag whether ot not adding of input rows should be thread-safe + * @param maxRowCount max number of rows */ protected IncrementalIndex( final IncrementalIndexSchema incrementalIndexSchema, - final boolean deserializeComplexMetrics, final boolean reportParseExceptions, - final boolean concurrentEventAdd + final int maxRowCount ) { this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.gran = incrementalIndexSchema.getGran(); this.rollup = incrementalIndexSchema.isRollup(); - this.virtualColumns = incrementalIndexSchema.getVirtualColumns(); - this.metrics = incrementalIndexSchema.getMetrics(); this.rowTransformers = new CopyOnWriteArrayList<>(); - this.deserializeComplexMetrics = deserializeComplexMetrics; this.reportParseExceptions = reportParseExceptions; + this.maxRowCount = maxRowCount; this.columnCapabilities = Maps.newHashMap(); this.metadata = new Metadata( null, - getCombiningAggregators(metrics), + AggsManager.getCombiningAggregators(incrementalIndexSchema.getMetrics()), incrementalIndexSchema.getTimestampSpec(), this.gran, this.rollup ); - this.aggs = initAggs(metrics, rowSupplier, deserializeComplexMetrics, concurrentEventAdd); - - this.metricDescs = Maps.newLinkedHashMap(); - for (AggregatorFactory metric : metrics) { - MetricDesc metricDesc = new MetricDesc(metricDescs.size(), metric); - metricDescs.put(metricDesc.getName(), metricDesc); - columnCapabilities.put(metricDesc.getName(), metricDesc.getCapabilities()); - } - DimensionsSpec dimensionsSpec = incrementalIndexSchema.getDimensionsSpec(); this.dimensionDescs = Maps.newLinkedHashMap(); @@ -427,7 +274,7 @@ public Builder setMaxBytesInMemory(final long maxBytesInMemory) return this; } - public IncrementalIndex buildOnheap() + public OnheapIncrementalIndex buildOnheap() { if (maxRowCount <= 0) { throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); @@ -460,6 +307,21 @@ public IncrementalIndex buildOffheap(final NonBlockingPool bufferPoo Objects.requireNonNull(bufferPool, "bufferPool is null") ); } + + public IncrementalIndex buildOffheapOak() + { + if (maxRowCount <= 0) { + throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); + } + + return new OakIncrementalIndex( + Objects.requireNonNull(incrementalIndexSchema, "incrementalIndexSchema is null"), + deserializeComplexMetrics, + reportParseExceptions, + concurrentEventAdd, + maxRowCount + ); + } } public boolean isRollup() @@ -467,48 +329,25 @@ public boolean isRollup() return rollup; } - public abstract FactsHolder getFacts(); - public abstract boolean canAppendRow(); public abstract String getOutOfRowsReason(); - protected abstract AggregatorType[] initAggs( - AggregatorFactory[] metrics, - Supplier rowSupplier, - boolean deserializeComplexMetrics, - boolean concurrentEventAdd - ); - - // Note: This method needs to be thread safe. - protected abstract AddToFactsResult addToFacts( - AggregatorFactory[] metrics, - boolean deserializeComplexMetrics, - boolean reportParseExceptions, - InputRow row, - AtomicInteger numEntries, - AtomicLong sizeInBytes, - IncrementalIndexRow key, - ThreadLocal rowContainer, - Supplier rowSupplier, - boolean skipMaxRowsInMemoryCheck - ) throws IndexSizeExceededException; - public abstract int getLastRowIndex(); - protected abstract AggregatorType[] getAggsForRow(int rowOffset); + protected abstract AggregatorType[] getAggsForRow(IncrementalIndexRow incrementalIndexRow); - protected abstract Object getAggVal(AggregatorType agg, int rowOffset, int aggPosition); + protected abstract Object getAggVal(IncrementalIndexRow incrementalIndexRow, int aggIndex); - protected abstract float getMetricFloatValue(int rowOffset, int aggOffset); + protected abstract float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggIndex); - protected abstract long getMetricLongValue(int rowOffset, int aggOffset); + protected abstract long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggIndex); - protected abstract Object getMetricObjectValue(int rowOffset, int aggOffset); + protected abstract Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggIndex); - protected abstract double getMetricDoubleValue(int rowOffset, int aggOffset); + protected abstract double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggIndex); - protected abstract boolean isNull(int rowOffset, int aggOffset); + protected abstract boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggIndex); static class IncrementalIndexRowResult { @@ -565,6 +404,12 @@ public List getParseExceptionMessages() } } + public abstract Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); + + public abstract Iterable keySet(); + + public abstract Iterable persistIterable(); + @Override public void close() { @@ -604,29 +449,7 @@ public IncrementalIndexAddResult add(InputRow row) throws IndexSizeExceededExcep return add(row, false); } - public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException - { - IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row); - final AddToFactsResult addToFactsResult = addToFacts( - metrics, - deserializeComplexMetrics, - reportParseExceptions, - row, - numEntries, - bytesInMemory, - incrementalIndexRowResult.getIncrementalIndexRow(), - in, - rowSupplier, - skipMaxRowsInMemoryCheck - ); - updateMaxIngestedTime(row.getTimestamp()); - ParseException parseException = getCombinedParseException( - row, - incrementalIndexRowResult.getParseExceptionMessages(), - addToFactsResult.getParseExceptionMessages() - ); - return new IncrementalIndexAddResult(addToFactsResult.getRowCount(), addToFactsResult.getBytesInMemory(), parseException); - } + public abstract IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException; @VisibleForTesting IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) @@ -768,7 +591,7 @@ public static ParseException getCombinedParseException( return pe; } - private synchronized void updateMaxIngestedTime(DateTime eventTime) + protected synchronized void updateMaxIngestedTime(DateTime eventTime) { if (maxIngestedEventTime == null || maxIngestedEventTime.isBefore(eventTime)) { maxIngestedEventTime = eventTime; @@ -790,25 +613,11 @@ public long getBytesInMemory() return bytesInMemory.get(); } - private long getMinTimeMillis() - { - return getFacts().getMinTimeMillis(); - } + protected abstract long getMinTimeMillis(); - private long getMaxTimeMillis() - { - return getFacts().getMaxTimeMillis(); - } + protected abstract long getMaxTimeMillis(); - public AggregatorType[] getAggs() - { - return aggs; - } - - public AggregatorFactory[] getMetricAggs() - { - return metrics; - } + public abstract AggregatorFactory[] getMetricAggs(); public List getDimensionNames() { @@ -833,34 +642,9 @@ public DimensionDesc getDimension(String dimension) } @Nullable - public String getMetricType(String metric) - { - final MetricDesc metricDesc = metricDescs.get(metric); - return metricDesc != null ? metricDesc.getType() : null; - } + public abstract String getMetricType(String metric); - public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) - { - MetricDesc metricDesc = metricDescs.get(metric); - if (metricDesc == null) { - return NilColumnValueSelector.instance(); - } - int metricIndex = metricDesc.getIndex(); - switch (metricDesc.getCapabilities().getType()) { - case COMPLEX: - return new ObjectMetricColumnSelector(metricDesc, currEntry, metricIndex); - case LONG: - return new LongMetricColumnSelector(currEntry, metricIndex); - case FLOAT: - return new FloatMetricColumnSelector(currEntry, metricIndex); - case DOUBLE: - return new DoubleMetricColumnSelector(currEntry, metricIndex); - case STRING: - throw new IllegalStateException("String is not a metric column type"); - default: - throw new ISE("Unknown metric value type: %s", metricDesc.getCapabilities().getType()); - } - } + public abstract ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry); public Interval getInterval() { @@ -934,10 +718,7 @@ private DimensionDesc addNewDimension(String dim, ColumnCapabilitiesImpl capabil return desc; } - public List getMetricNames() - { - return ImmutableList.copyOf(metricDescs.keySet()); - } + public abstract List getMetricNames(); @Override public List getColumnNames() @@ -963,72 +744,14 @@ public Metadata getMetadata() return metadata; } - private static AggregatorFactory[] getCombiningAggregators(AggregatorFactory[] aggregators) - { - AggregatorFactory[] combiningAggregators = new AggregatorFactory[aggregators.length]; - for (int i = 0; i < aggregators.length; i++) { - combiningAggregators[i] = aggregators[i].getCombiningFactory(); - } - return combiningAggregators; - } - @Override public Iterator iterator() { return iterableWithPostAggregations(null, false).iterator(); } - public Iterable iterableWithPostAggregations(final List postAggs, final boolean descending) - { - return new Iterable() - { - @Override - public Iterator iterator() - { - final List dimensions = getDimensions(); - - return Iterators.transform( - getFacts().iterator(descending), - incrementalIndexRow -> { - final int rowOffset = incrementalIndexRow.getRowIndex(); - - Object[] theDims = incrementalIndexRow.getDims(); - - Map theVals = Maps.newLinkedHashMap(); - for (int i = 0; i < theDims.length; ++i) { - Object dim = theDims[i]; - DimensionDesc dimensionDesc = dimensions.get(i); - if (dimensionDesc == null) { - continue; - } - String dimensionName = dimensionDesc.getName(); - DimensionHandler handler = dimensionDesc.getHandler(); - if (dim == null || handler.getLengthOfEncodedKeyComponent(dim) == 0) { - theVals.put(dimensionName, null); - continue; - } - final DimensionIndexer indexer = dimensionDesc.getIndexer(); - Object rowVals = indexer.convertUnsortedEncodedKeyComponentToActualList(dim); - theVals.put(dimensionName, rowVals); - } - - AggregatorType[] aggs = getAggsForRow(rowOffset); - for (int i = 0; i < aggs.length; ++i) { - theVals.put(metrics[i].getName(), getAggVal(aggs[i], rowOffset, i)); - } - - if (postAggs != null) { - for (PostAggregator postAgg : postAggs) { - theVals.put(postAgg.getName(), postAgg.compute(theVals)); - } - } - - return new MapBasedRow(incrementalIndexRow.getTimestamp(), theVals); - } - ); - } - }; - } + public abstract Iterable iterableWithPostAggregations(List postAggs, boolean descending); + public DateTime getMaxIngestedEventTime() { @@ -1078,65 +801,6 @@ public DimensionIndexer getIndexer() } } - public static final class MetricDesc - { - private final int index; - private final String name; - private final String type; - private final ColumnCapabilitiesImpl capabilities; - - public MetricDesc(int index, AggregatorFactory factory) - { - this.index = index; - this.name = factory.getName(); - - String typeInfo = factory.getTypeName(); - this.capabilities = new ColumnCapabilitiesImpl(); - if ("float".equalsIgnoreCase(typeInfo)) { - capabilities.setType(ValueType.FLOAT); - this.type = typeInfo; - } else if ("long".equalsIgnoreCase(typeInfo)) { - capabilities.setType(ValueType.LONG); - this.type = typeInfo; - } else if ("double".equalsIgnoreCase(typeInfo)) { - capabilities.setType(ValueType.DOUBLE); - this.type = typeInfo; - } else { - capabilities.setType(ValueType.COMPLEX); - this.type = ComplexMetrics.getSerdeForType(typeInfo).getTypeName(); - } - } - - public int getIndex() - { - return index; - } - - public String getName() - { - return name; - } - - public String getType() - { - return type; - } - - public ColumnCapabilitiesImpl getCapabilities() - { - return capabilities; - } - } - - protected ColumnSelectorFactory makeColumnSelectorFactory( - final AggregatorFactory agg, - final Supplier in, - final boolean deserializeComplexMetrics - ) - { - return makeColumnSelectorFactory(virtualColumns, agg, in, deserializeComplexMetrics); - } - protected final Comparator dimsComparator() { return new IncrementalIndexRowComparator(dimensionDescsList); @@ -1155,13 +819,13 @@ public IncrementalIndexRowComparator(List dimDescs) @Override public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) { - int retVal = Longs.compare(lhs.timestamp, rhs.timestamp); - int numComparisons = Math.min(lhs.dims.length, rhs.dims.length); + int retVal = Longs.compare(lhs.getTimestamp(), rhs.getTimestamp()); + int numComparisons = Math.min(lhs.getDimsLength(), rhs.getDimsLength()); int index = 0; while (retVal == 0 && index < numComparisons) { - final Object lhsIdxs = lhs.dims[index]; - final Object rhsIdxs = rhs.dims[index]; + final Object lhsIdxs = lhs.getDim(index); + final Object rhsIdxs = rhs.getDim(index); if (lhsIdxs == null) { if (rhsIdxs == null) { @@ -1181,11 +845,11 @@ public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) } if (retVal == 0) { - int lengthDiff = Ints.compare(lhs.dims.length, rhs.dims.length); + int lengthDiff = Ints.compare(lhs.getDimsLength(), rhs.getDimsLength()); if (lengthDiff == 0) { return 0; } - Object[] largerDims = lengthDiff > 0 ? lhs.dims : rhs.dims; + IncrementalIndexRow largerDims = lengthDiff > 0 ? lhs : rhs; return allNull(largerDims, numComparisons) ? 0 : lengthDiff; } @@ -1193,399 +857,13 @@ public int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) } } - private static boolean allNull(Object[] dims, int startPosition) + public static boolean allNull(IncrementalIndexRow row, int startPosition) { - for (int i = startPosition; i < dims.length; i++) { - if (dims[i] != null) { + for (int i = startPosition; i < row.getDimsLength(); i++) { + if (row.getDim(i) != null) { return false; } } return true; } - - interface FactsHolder - { - /** - * @return the previous rowIndex associated with the specified key, or - * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. - */ - int getPriorIndex(IncrementalIndexRow key); - - long getMinTimeMillis(); - - long getMaxTimeMillis(); - - Iterator iterator(boolean descending); - - Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd); - - Iterable keySet(); - - /** - * Get all {@link IncrementalIndexRow} to persist, ordered with {@link Comparator} - * @return - */ - Iterable persistIterable(); - - /** - * @return the previous rowIndex associated with the specified key, or - * {@link IncrementalIndexRow#EMPTY_ROW_INDEX} if there was no mapping for the key. - */ - int putIfAbsent(IncrementalIndexRow key, int rowIndex); - - void clear(); - } - - static class RollupFactsHolder implements FactsHolder - { - private final boolean sortFacts; - // Can't use Set because we need to be able to get from collection - private final ConcurrentMap facts; - private final List dimensionDescsList; - - RollupFactsHolder( - boolean sortFacts, - Comparator incrementalIndexRowComparator, - List dimensionDescsList - ) - { - this.sortFacts = sortFacts; - if (sortFacts) { - this.facts = new ConcurrentSkipListMap<>(incrementalIndexRowComparator); - } else { - this.facts = new ConcurrentHashMap<>(); - } - this.dimensionDescsList = dimensionDescsList; - } - - @Override - public int getPriorIndex(IncrementalIndexRow key) - { - IncrementalIndexRow row = facts.get(key); - return row == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : row.getRowIndex(); - } - - @Override - public long getMinTimeMillis() - { - if (sortFacts) { - return ((ConcurrentNavigableMap) facts).firstKey().getTimestamp(); - } else { - throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); - } - } - - @Override - public long getMaxTimeMillis() - { - if (sortFacts) { - return ((ConcurrentNavigableMap) facts).lastKey().getTimestamp(); - } else { - throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); - } - } - - @Override - public Iterator iterator(boolean descending) - { - if (descending && sortFacts) { - return ((ConcurrentNavigableMap) facts).descendingMap().keySet().iterator(); - } - return keySet().iterator(); - } - - @Override - public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) - { - if (!sortFacts) { - throw new UnsupportedOperationException("can't get timeRange from unsorted facts data."); - } - IncrementalIndexRow start = new IncrementalIndexRow(timeStart, new Object[]{}, dimensionDescsList); - IncrementalIndexRow end = new IncrementalIndexRow(timeEnd, new Object[]{}, dimensionDescsList); - ConcurrentNavigableMap subMap = - ((ConcurrentNavigableMap) facts).subMap(start, end); - final Map rangeMap = descending ? subMap.descendingMap() : subMap; - return rangeMap.keySet(); - } - - @Override - public Iterable keySet() - { - return facts.keySet(); - } - - @Override - public Iterable persistIterable() - { - // with rollup, facts are already pre-sorted so just return keyset - return keySet(); - } - - @Override - public int putIfAbsent(IncrementalIndexRow key, int rowIndex) - { - // setRowIndex() must be called before facts.putIfAbsent() for visibility of rowIndex from concurrent readers. - key.setRowIndex(rowIndex); - IncrementalIndexRow prev = facts.putIfAbsent(key, key); - return prev == null ? IncrementalIndexRow.EMPTY_ROW_INDEX : prev.getRowIndex(); - } - - @Override - public void clear() - { - facts.clear(); - } - } - - static class PlainFactsHolder implements FactsHolder - { - private final boolean sortFacts; - private final ConcurrentMap> facts; - - private final Comparator incrementalIndexRowComparator; - - public PlainFactsHolder(boolean sortFacts, Comparator incrementalIndexRowComparator) - { - this.sortFacts = sortFacts; - if (sortFacts) { - this.facts = new ConcurrentSkipListMap<>(); - } else { - this.facts = new ConcurrentHashMap<>(); - } - this.incrementalIndexRowComparator = incrementalIndexRowComparator; - } - - @Override - public int getPriorIndex(IncrementalIndexRow key) - { - // always return EMPTY_ROW_INDEX to indicate that no prior key cause we always add new row - return IncrementalIndexRow.EMPTY_ROW_INDEX; - } - - @Override - public long getMinTimeMillis() - { - if (sortFacts) { - return ((ConcurrentNavigableMap>) facts).firstKey(); - } else { - throw new UnsupportedOperationException("can't get minTime from unsorted facts data."); - } - } - - @Override - public long getMaxTimeMillis() - { - if (sortFacts) { - return ((ConcurrentNavigableMap>) facts).lastKey(); - } else { - throw new UnsupportedOperationException("can't get maxTime from unsorted facts data."); - } - } - - @Override - public Iterator iterator(boolean descending) - { - if (descending && sortFacts) { - return timeOrderedConcat(((ConcurrentNavigableMap>) facts) - .descendingMap().values(), true).iterator(); - } - return timeOrderedConcat(facts.values(), false).iterator(); - } - - @Override - public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) - { - ConcurrentNavigableMap> subMap = - ((ConcurrentNavigableMap>) facts).subMap(timeStart, timeEnd); - final Map> rangeMap = descending ? subMap.descendingMap() : subMap; - return timeOrderedConcat(rangeMap.values(), descending); - } - - private Iterable timeOrderedConcat( - final Iterable> iterable, - final boolean descending - ) - { - return () -> Iterators.concat( - Iterators.transform( - iterable.iterator(), - input -> descending ? input.descendingIterator() : input.iterator() - ) - ); - } - - private Stream timeAndDimsOrderedConcat( - final Collection> rowGroups - ) - { - return rowGroups.stream() - .flatMap(Collection::stream) - .sorted(incrementalIndexRowComparator); - } - - @Override - public Iterable keySet() - { - return timeOrderedConcat(facts.values(), false); - } - - @Override - public Iterable persistIterable() - { - return () -> timeAndDimsOrderedConcat(facts.values()).iterator(); - } - - @Override - public int putIfAbsent(IncrementalIndexRow key, int rowIndex) - { - Long time = key.getTimestamp(); - Deque rows = facts.get(time); - if (rows == null) { - facts.putIfAbsent(time, new ConcurrentLinkedDeque<>()); - // in race condition, rows may be put by other thread, so always get latest status from facts - rows = facts.get(time); - } - // setRowIndex() must be called before rows.add() for visibility of rowIndex from concurrent readers. - key.setRowIndex(rowIndex); - rows.add(key); - // always return EMPTY_ROW_INDEX to indicate that we always add new row - return IncrementalIndexRow.EMPTY_ROW_INDEX; - } - - @Override - public void clear() - { - facts.clear(); - } - } - - private class LongMetricColumnSelector implements LongColumnSelector - { - private final IncrementalIndexRowHolder currEntry; - private final int metricIndex; - - public LongMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) - { - this.currEntry = currEntry; - this.metricIndex = metricIndex; - } - - @Override - public long getLong() - { - assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricLongValue(currEntry.get().getRowIndex(), metricIndex); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("index", IncrementalIndex.this); - } - - @Override - public boolean isNull() - { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); - } - } - - private class ObjectMetricColumnSelector extends ObjectColumnSelector - { - private final IncrementalIndexRowHolder currEntry; - private final int metricIndex; - private Class classOfObject; - - public ObjectMetricColumnSelector( - MetricDesc metricDesc, - IncrementalIndexRowHolder currEntry, - int metricIndex - ) - { - this.currEntry = currEntry; - this.metricIndex = metricIndex; - classOfObject = ComplexMetrics.getSerdeForType(metricDesc.getType()).getObjectStrategy().getClazz(); - } - - @Nullable - @Override - public Object getObject() - { - return getMetricObjectValue(currEntry.get().getRowIndex(), metricIndex); - } - - @Override - public Class classOfObject() - { - return classOfObject; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("index", IncrementalIndex.this); - } - } - - private class FloatMetricColumnSelector implements FloatColumnSelector - { - private final IncrementalIndexRowHolder currEntry; - private final int metricIndex; - - public FloatMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) - { - this.currEntry = currEntry; - this.metricIndex = metricIndex; - } - - @Override - public float getFloat() - { - assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricFloatValue(currEntry.get().getRowIndex(), metricIndex); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("index", IncrementalIndex.this); - } - - @Override - public boolean isNull() - { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); - } - } - - private class DoubleMetricColumnSelector implements DoubleColumnSelector - { - private final IncrementalIndexRowHolder currEntry; - private final int metricIndex; - - public DoubleMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricIndex) - { - this.currEntry = currEntry; - this.metricIndex = metricIndex; - } - - @Override - public double getDouble() - { - assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricDoubleValue(currEntry.get().getRowIndex(), metricIndex); - } - - @Override - public boolean isNull() - { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("index", IncrementalIndex.this); - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java index 2493bdd391e6..3709d6c1b21f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexAdapter.java @@ -93,15 +93,16 @@ private void processRows( ) { int rowNum = 0; - for (IncrementalIndexRow row : index.getFacts().persistIterable()) { - final Object[] dims = row.getDims(); + for (IncrementalIndexRow row : index.persistIterable()) { for (IncrementalIndex.DimensionDesc dimension : dimensions) { final int dimIndex = dimension.getIndex(); DimensionAccessor accessor = accessors.get(dimension.getName()); + Object dim = row.getDim(dimIndex); + // Add 'null' to the dimension's dictionary. - if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (dim == null) { accessor.indexer.processRowValsToUnsortedEncodedKeyComponent(null, true); continue; } @@ -110,7 +111,7 @@ private void processRows( if (capabilities.hasBitmapIndexes()) { final MutableBitmap[] bitmapIndexes = accessor.invertedIndexes; final DimensionIndexer indexer = accessor.indexer; - indexer.fillBitmapsFromUnsortedEncodedKeyComponent(dims[dimIndex], rowNum, bitmapIndexes, bitmapFactory); + indexer.fillBitmapsFromUnsortedEncodedKeyComponent(dim, rowNum, bitmapIndexes, bitmapFactory); } } ++rowNum; 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 5c4d7b613ec0..a3fa667fb3a9 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 @@ -30,23 +30,25 @@ import java.util.Collections; import java.util.List; -public final class IncrementalIndexRow +public class IncrementalIndexRow { public static final int EMPTY_ROW_INDEX = -1; - final long timestamp; - final Object[] dims; - private final List dimensionDescsList; + private long timestamp; + private Object[] dims; + protected List dimensionDescsList; /** * rowIndex is not checked in {@link #equals} and {@link #hashCode} on purpose. IncrementalIndexRow acts as a Map key * and "entry" object (rowIndex is the "value") at the same time. This is done to reduce object indirection and * improve locality, and avoid boxing of rowIndex as Integer, when stored in JDK collection: - * {@link IncrementalIndex.RollupFactsHolder} needs concurrent collections, that are not present in fastutil. + * {@link ExternalDataIncrementalIndex.RollupFactsHolder} needs concurrent collections, that are not present in fastutil. */ private int rowIndex; private long dimsKeySize; + protected IncrementalIndexRow() {} + IncrementalIndexRow( long timestamp, Object[] dims, @@ -69,7 +71,7 @@ public final class IncrementalIndexRow this.rowIndex = rowIndex; } - private IncrementalIndexRow( + protected IncrementalIndexRow( long timestamp, Object[] dims, List dimensionDescsList, @@ -97,6 +99,19 @@ public long getTimestamp() return timestamp; } + public int getDimsLength() + { + return dims == null ? 0 : dims.length; + } + + public Object getDim(int dimIndex) + { + if (dims == null || dimIndex >= dims.length) { + return null; + } + return dims[dimIndex]; + } + public Object[] getDims() { return dims; @@ -112,6 +127,33 @@ void setRowIndex(int rowIndex) this.rowIndex = rowIndex; } + public int copyStringDim(int dimIndex, int[] dst) + { + if (dims == null || dimIndex >= dims.length) { + return 0; + } + int[] stringDim = (int[]) dims[dimIndex]; + if (stringDim != null) { + if (dst.length < stringDim.length) { + dst = new int[stringDim.length]; + } + System.arraycopy((int[]) dims[dimIndex], 0, dst, 0, stringDim.length); + return stringDim.length; + } else { + return 0; + } + } + + public int calcStringDimSize(int dimIndex) + { + if (dims == null || dimIndex >= dims.length || dims[dimIndex] == null) { + return 0; + } + + return ((int[]) dims[dimIndex]).length; + + } + /** * 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: @@ -135,19 +177,19 @@ public long estimateBytesInMemory() 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"); + "timestamp=" + DateTimes.utc(getTimestamp()) + + ", dims=" + Lists.transform( + Arrays.asList(getDims()), new Function() + { + @Override + public Object apply(@Nullable Object input) + { + if (input == null || Array.getLength(input) == 0) { + return Collections.singletonList("null"); + } + return Collections.singletonList(input); + } } - return Collections.singletonList(input); - } - } ) + '}'; } @@ -163,15 +205,15 @@ public boolean equals(Object o) IncrementalIndexRow that = (IncrementalIndexRow) o; - if (timestamp != that.timestamp) { + if (getTimestamp() != that.getTimestamp()) { return false; } - if (dims.length != that.dims.length) { + if (getDimsLength() != that.getDimsLength()) { return false; } - for (int i = 0; i < dims.length; i++) { + for (int i = 0; i < getDimsLength(); i++) { final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); - if (!indexer.checkUnsortedEncodedKeyComponentsEqual(dims[i], that.dims[i])) { + if (!indexer.checkUnsortedEncodedKeyComponentsEqual(getDim(i), that.getDim(i))) { return false; } } @@ -181,8 +223,8 @@ public boolean equals(Object o) @Override public int hashCode() { - int hash = (int) timestamp; - for (int i = 0; i < dims.length; i++) { + int hash = (int) getTimestamp(); + for (int i = 0; i < getDimsLength(); i++) { final DimensionIndexer indexer = dimensionDescsList.get(i).getIndexer(); hash = 31 * hash + indexer.getUnsortedEncodedKeyComponentHashCode(dims[i]); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java index 627c63f2d8a7..cc0dba645275 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRowIterator.java @@ -50,7 +50,7 @@ class IncrementalIndexRowIterator implements TransformableRowIterator IncrementalIndexRowIterator(IncrementalIndex incrementalIndex) { - this.timeAndDimsIterator = incrementalIndex.getFacts().persistIterable().iterator(); + this.timeAndDimsIterator = incrementalIndex.persistIterable().iterator(); this.currentRowPointer = makeRowPointer(incrementalIndex, currentRowHolder, currentRowNumCounter); // markedRowPointer doesn't actually need to be a RowPointer (just a TimeAndDimsPointer), but we create a RowPointer // in order to reuse the makeRowPointer() method. Passing a dummy RowNumCounter. diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index ae8e14a2aad6..78495b31037f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -243,7 +243,7 @@ private class IncrementalIndexCursor implements Cursor filterMatcher = filter == null ? BooleanValueMatcher.of(true) : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); - cursorIterable = index.getFacts().timeRangeIterable( + cursorIterable = index.timeRangeIterable( descending, timeStart, Math.min(actualInterval.getEndMillis(), gran.increment(interval.getStart()).getMillis()) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/InternalDataIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/InternalDataIncrementalIndex.java new file mode 100644 index 000000000000..80050a8b9ecb --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/InternalDataIncrementalIndex.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + + +public abstract class InternalDataIncrementalIndex extends IncrementalIndex +{ + + protected InternalDataIncrementalIndex( + IncrementalIndexSchema incrementalIndexSchema, + boolean reportParseExceptions, + int maxRowCount + ) + { + super(incrementalIndexSchema, reportParseExceptions, maxRowCount); + } + + @Override + public String getOutOfRowsReason() + { + return outOfRowsReason; + } + +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakComputer.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakComputer.java new file mode 100644 index 000000000000..18a162c01c06 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakComputer.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import org.apache.druid.data.input.InputRow; + +import java.nio.ByteBuffer; +import java.util.function.Consumer; + + +/** + * For sending as input parameter to the Oak's putifAbsentComputeIfPresent method + */ +public class OakComputer implements Consumer +{ + OffheapAggsManager aggsManager; + boolean reportParseExceptions; + InputRow row; + ThreadLocal rowContainer; + + public OakComputer( + OffheapAggsManager aggsManager, + boolean reportParseExceptions, + InputRow row, + ThreadLocal rowContainer + ) + { + this.aggsManager = aggsManager; + this.reportParseExceptions = reportParseExceptions; + this.row = row; + this.rowContainer = rowContainer; + } + + @Override + public void accept(ByteBuffer byteBuffer) + { + aggsManager.aggregate(reportParseExceptions, row, rowContainer, byteBuffer); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java new file mode 100644 index 000000000000..ca6bb9d8881c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java @@ -0,0 +1,537 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import com.google.common.collect.Iterators; +import com.google.common.collect.Maps; +import com.oath.oak.OakIterator; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.segment.ColumnValueSelector; +import com.oath.oak.OakMap; +import com.oath.oak.OakMapBuilder; +import com.oath.oak.OakTransformView; +import com.oath.oak.OakBufferView; +import com.oath.oak.OakRBuffer; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.Map; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +import org.apache.druid.segment.DimensionHandler; +import org.apache.druid.segment.DimensionIndexer; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ValueType; +import sun.misc.VM; + +import javax.annotation.Nullable; + + +/** + */ +public class OakIncrementalIndex extends InternalDataIncrementalIndex +{ + + // When serializing an object from IncrementalIndexRow.dims, we use: + // 1. 4 bytes for representing its type (Double, Float, Long or String) + // 2. 8 bytes for saving its value or the array position and length (in the case of String) + static final Integer ALLOC_PER_DIM = 12; + static final Integer NO_DIM = -1; + static final Integer TIME_STAMP_INDEX = 0; + static final Integer DIMS_LENGTH_INDEX = TIME_STAMP_INDEX + Long.BYTES; + static final Integer ROW_INDEX_INDEX = DIMS_LENGTH_INDEX + Integer.BYTES; + static final Integer DIMS_INDEX = ROW_INDEX_INDEX + Integer.BYTES; + // Serialization and deserialization offsets + static final Integer VALUE_TYPE_OFFSET = 0; + static final Integer DATA_OFFSET = VALUE_TYPE_OFFSET + Integer.BYTES; + static final Integer ARRAY_INDEX_OFFSET = VALUE_TYPE_OFFSET + Integer.BYTES; + static final Integer ARRAY_LENGTH_OFFSET = ARRAY_INDEX_OFFSET + Integer.BYTES; + + OakMap oak; + + private OffheapAggsManager aggsManager; + private AtomicInteger rowIndexGenerator; // Only used in Plain mode + + Map env = System.getenv(); + + OakIncrementalIndex( + IncrementalIndexSchema incrementalIndexSchema, + boolean deserializeComplexMetrics, + boolean reportParseExceptions, + boolean concurrentEventAdd, + int maxRowCount + ) + { + super(incrementalIndexSchema, reportParseExceptions, maxRowCount); + + this.aggsManager = new OffheapAggsManager(incrementalIndexSchema, deserializeComplexMetrics, + concurrentEventAdd, rowSupplier, + columnCapabilities, null, this); + + this.rowIndexGenerator = new AtomicInteger(IncrementalIndexRow.EMPTY_ROW_INDEX); + IncrementalIndexRow minIncrementalIndexRow = getMinIncrementalIndexRow(); + + long maxDirectMemory = VM.maxDirectMemory(); + int memoryCapacity = Integer.MAX_VALUE; + if (maxDirectMemory < memoryCapacity) { + memoryCapacity = (int) maxDirectMemory; + } + + OakMapBuilder builder = new OakMapBuilder() + .setKeySerializer(new OakKeySerializer(dimensionDescsList)) + .setValueSerializer(new OakValueSerializer(aggsManager, reportParseExceptions, in)) + .setMinKey(minIncrementalIndexRow) + .setComparator(new OakKeysComparator(dimensionDescsList, this.isRollup())) + .setMemoryCapacity(memoryCapacity); + + if (env != null) { + String chunkMaxItems = env.get("chunkMaxItems"); + if (chunkMaxItems != null) { + builder = builder.setChunkMaxItems(Integer.getInteger(chunkMaxItems)); + } + String chunkBytesPerItem = env.get("chunkBytesPerItem"); + if (chunkMaxItems != null) { + builder = builder.setChunkBytesPerItem(Integer.getInteger(chunkBytesPerItem)); + } + } + + oak = builder.build(); + } + + @Override + public int getLastRowIndex() + { + return this.rowIndexGenerator.get(); + } + + @Override + public Iterable iterableWithPostAggregations(List postAggs, boolean descending) + { + Function, Row> transformer = new Function, Row>() + { + + @Override + public Row apply(Map.Entry entry) + { + ByteBuffer serializedKey = entry.getKey(); + ByteBuffer serializedValue = entry.getValue(); + long timeStamp = OakIncrementalIndex.getTimestamp(serializedKey); + int dimsLength = OakIncrementalIndex.getDimsLength(serializedKey); + Map theVals = Maps.newLinkedHashMap(); + for (int i = 0; i < dimsLength; ++i) { + Object dim = OakIncrementalIndex.getDimValue(serializedKey, i); + DimensionDesc dimensionDesc = dimensionDescsList.get(i); + if (dimensionDesc == null) { + continue; + } + String dimensionName = dimensionDesc.getName(); + DimensionHandler handler = dimensionDesc.getHandler(); + if (dim == null || handler.getLengthOfEncodedKeyComponent(dim) == 0) { + theVals.put(dimensionName, null); + continue; + } + final DimensionIndexer indexer = dimensionDesc.getIndexer(); + Object rowVals = indexer.convertUnsortedEncodedKeyComponentToActualList(dim); + theVals.put(dimensionName, rowVals); + } + + BufferAggregator[] aggs = aggsManager.getAggs(); + for (int i = 0; i < aggs.length; ++i) { + theVals.put(aggsManager.metrics[i].getName(), aggs[i].get(serializedValue, aggsManager.aggOffsetInBuffer[i])); + } + + return new MapBasedRow(timeStamp, theVals); + } + }; + + + + try (OakMap tmpOakMap = descending ? oak.descendingMap() : oak; + OakTransformView transformView = tmpOakMap.createTransformView(transformer)) { + OakIterator valuesIterator = transformView.entriesIterator(); + return () -> Iterators.transform( + valuesIterator, + row -> row + ); + } + } + + @Override + public Iterable persistIterable() + { + return keySet(); + } + + @Override + public void close() + { + oak.close(); + } + + @Override + protected long getMinTimeMillis() + { + return oak.getMinKey().getTimestamp(); + } + + @Override + protected long getMaxTimeMillis() + { + return oak.getMaxKey().getTimestamp(); + } + + @Override + protected BufferAggregator[] getAggsForRow(IncrementalIndexRow incrementalIndexRow) + { + return aggsManager.getAggs(); + } + + @Override + protected Object getAggVal(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Object> transformer = entry -> { + ByteBuffer serializedValue = entry.getValue(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.get(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); + }; + + try (OakTransformView transformView = oak.createTransformView(transformer)) { + return transformView.get(incrementalIndexRow); + } + } + + @Override + protected float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Float> transformer = new Function, Float>() { + @Override + public Float apply(Map.Entry entry) + { + ByteBuffer serializedValue = entry.getValue(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.getFloat(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); + } + }; + + try (OakTransformView transformView = oak.createTransformView(transformer)) { + return transformView.get(incrementalIndexRow); + } + } + + @Override + protected long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Long> transformer = new Function, Long>() { + @Override + public Long apply(Map.Entry entry) + { + ByteBuffer serializedValue = entry.getValue(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.getLong(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); + } + }; + + try (OakTransformView transformView = oak.createTransformView(transformer)) { + return transformView.get(incrementalIndexRow); + } + } + + @Override + protected Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Object> transformer = new Function, Object>() { + @Override + public Object apply(Map.Entry entry) + { + ByteBuffer serializedValue = entry.getValue(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.get(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); + } + }; + + try (OakTransformView transformView = oak.createTransformView(transformer)) { + return transformView.get(incrementalIndexRow); + } + } + + @Override + protected double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Double> transformer = new Function, Double>() { + @Override + public Double apply(Map.Entry entry) + { + ByteBuffer serializedValue = entry.getValue(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.getDouble(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); + } + }; + try (OakTransformView transformView = oak.createTransformView(transformer)) { + return transformView.get(incrementalIndexRow); + } + } + + @Override + protected boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Boolean> transformer = new Function, Boolean>() { + @Override + public Boolean apply(Map.Entry entry) + { + ByteBuffer serializedValue = entry.getValue(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.isNull(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); + } + }; + + try (OakTransformView transformView = oak.createTransformView(transformer)) { + return transformView.get(incrementalIndexRow); + } + } + + @Override + public Iterable timeRangeIterable( + boolean descending, long timeStart, long timeEnd) + { + if (timeStart > timeEnd) { + return null; + } + + IncrementalIndexRow from = new IncrementalIndexRow(timeStart, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); + IncrementalIndexRow to = new IncrementalIndexRow(timeEnd + 1, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); + try (OakMap subMap = oak.subMap(from, true, to, false, descending); + OakBufferView bufferView = subMap.createBufferView()) { + + OakIterator keysIterator = bufferView.keysIterator(); + return () -> Iterators.transform( + keysIterator, + byteBuffer -> new OakIncrementalIndexRow(byteBuffer, dimensionDescsList) + ); + + + } + + } + + @Override + public Iterable keySet() + { + OakIterator keysIterator = oak.keysIterator(); + + return new Iterable() { + @Override + public Iterator iterator() + { + return Iterators.transform( + keysIterator, + key -> key + ); + } + }; + } + + @Override + public boolean canAppendRow() + { + final boolean canAdd = size() < maxRowCount; + if (!canAdd) { + outOfRowsReason = StringUtils.format("Maximum number of rows [%d] reached", maxRowCount); + } + return canAdd; + } + + private Integer addToOak( + InputRow row, + AtomicInteger numEntries, + IncrementalIndexRow incrementalIndexRow, + ThreadLocal rowContainer, + boolean skipMaxRowsInMemoryCheck + ) throws IndexSizeExceededException + { + OakComputer computer = new OakComputer(aggsManager, reportParseExceptions, + row, rowContainer); + + if (isRollup()) { + incrementalIndexRow.setRowIndex(IncrementalIndexRow.EMPTY_ROW_INDEX); + } else { + incrementalIndexRow.setRowIndex(rowIndexGenerator.getAndIncrement()); + } + + if (numEntries.get() < maxRowCount || skipMaxRowsInMemoryCheck) { + oak.putIfAbsentComputeIfPresent(incrementalIndexRow, row, computer); + + int currSize = oak.entries(); + int prev = numEntries.get(); + while (currSize > prev) { + if (numEntries.compareAndSet(prev, currSize)) { + break; + } + prev = numEntries.get(); + } + + } else { + if (!oak.computeIfPresent(incrementalIndexRow, computer)) { // the key wasn't in oak + throw new IndexSizeExceededException("Maximum number of rows [%d] reached", maxRowCount); + } + } + return numEntries.get(); + } + + @Override + public AggregatorFactory[] getMetricAggs() + { + return aggsManager.getMetricAggs(); + } + + @Override + public IncrementalIndexAddResult add(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException + { + IncrementalIndexRowResult incrementalIndexRowResult = toIncrementalIndexRow(row); + final int rv = addToOak( + row, + numEntries, + incrementalIndexRowResult.getIncrementalIndexRow(), + in, + skipMaxRowsInMemoryCheck + ); + updateMaxIngestedTime(row.getTimestamp()); + return new IncrementalIndexAddResult(rv, 0, null); + } + + private IncrementalIndexRow getMinIncrementalIndexRow() + { + return new IncrementalIndexRow(this.minTimestamp, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); + } + + @Nullable + @Override + public String getMetricType(String metric) + { + return aggsManager.getMetricType(metric); + } + + @Override + public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) + { + return aggsManager.makeMetricColumnValueSelector(metric, currEntry); + } + + @Override + public List getMetricNames() + { + return aggsManager.getMetricNames(); + } + + /* ---------------- Serialization utils -------------- */ + + static long getTimestamp(ByteBuffer buff) + { + return buff.getLong(buff.position() + TIME_STAMP_INDEX); + } + + static int getDimsLength(ByteBuffer buff) + { + return buff.getInt(buff.position() + DIMS_LENGTH_INDEX); + } + + static int getDimIndexInBuffer(ByteBuffer buff, int dimsLength, int dimIndex) + { + if (dimIndex >= dimsLength) { + return NO_DIM; + } + return buff.position() + DIMS_INDEX + dimIndex * ALLOC_PER_DIM; + } + + static Object getDimValue(ByteBuffer buff, int dimIndex) + { + int dimsLength = getDimsLength(buff); + return getDimValue(buff, dimIndex, dimsLength); + } + + static Object getDimValue(ByteBuffer buff, int dimIndex, int dimsLength) + { + Object dimObject = null; + if (dimIndex >= dimsLength) { + return null; + } + int dimIndexInBuffer = getDimIndexInBuffer(buff, dimsLength, dimIndex); + int dimType = buff.getInt(dimIndexInBuffer); + if (dimType == OakIncrementalIndex.NO_DIM) { + return null; + } else if (dimType == ValueType.DOUBLE.ordinal()) { + dimObject = buff.getDouble(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.FLOAT.ordinal()) { + dimObject = buff.getFloat(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.LONG.ordinal()) { + dimObject = buff.getLong(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.STRING.ordinal()) { + int arrayIndexOffset = buff.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_INDEX_OFFSET); + int arrayIndex = buff.position() + arrayIndexOffset; + int arraySize = buff.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + int[] array = new int[arraySize]; + for (int i = 0; i < arraySize; i++) { + array[i] = buff.getInt(arrayIndex); + arrayIndex += Integer.BYTES; + } + dimObject = array; + } + + return dimObject; + } + + + static boolean checkDimsAllNull(ByteBuffer buff, int numComparisons) + { + int dimsLength = getDimsLength(buff); + for (int index = 0; index < Math.min(dimsLength, numComparisons); index++) { + if (buff.getInt(getDimIndexInBuffer(buff, dimsLength, index)) != OakIncrementalIndex.NO_DIM) { + return false; + } + } + return true; + } + + static ValueType getDimValueType(int dimIndex, List dimensionDescsList) + { + DimensionDesc dimensionDesc = dimensionDescsList.get(dimIndex); + if (dimensionDesc == null) { + return null; + } + ColumnCapabilitiesImpl capabilities = dimensionDesc.getCapabilities(); + if (capabilities == null) { + return null; + } + return capabilities.getType(); + } + + static int getRowIndex(ByteBuffer buff) + { + return buff.getInt(buff.position() + ROW_INDEX_INDEX); + } + +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java new file mode 100644 index 000000000000..ac72d188338d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + + +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.incremental.IncrementalIndex.DimensionDesc; + +import java.nio.ByteBuffer; +import java.util.List; + +public class OakIncrementalIndexRow extends IncrementalIndexRow +{ + static final int NO_LENGTH = -1; + + ByteBuffer oakRBuffer; + int dimsLength; + + public OakIncrementalIndexRow(ByteBuffer oakRBuffer, List dimensionDescsList) + { + this.oakRBuffer = oakRBuffer; + this.dimensionDescsList = dimensionDescsList; + this.dimsLength = NO_LENGTH; // lazy initialization + } + + @Override + public long getTimestamp() + { + return oakRBuffer.getLong(oakRBuffer.position() + OakIncrementalIndex.TIME_STAMP_INDEX); + } + + @Override + public int getDimsLength() + { + if (dimsLength == NO_LENGTH) { + dimsLength = oakRBuffer.getInt(oakRBuffer.position() + OakIncrementalIndex.DIMS_LENGTH_INDEX); + } + return dimsLength; + } + + @Override + public Object getDim(int dimIndex) + { + if (dimIndex >= getDimsLength()) { + return null; + } + return getDimValue(dimIndex); + } + + @Override + public Object[] getDims() + { + if (getDimsLength() == 0) { + return null; + } + Object[] dims = new Object[getDimsLength()]; + for (int dimIndex = 0; dimIndex < getDimsLength(); dimIndex++) { + Object dim = getDimValue(dimIndex); + dims[dimIndex] = dim; + } + return dims; + } + + @Override + public int getRowIndex() + { + return oakRBuffer.getInt(oakRBuffer.position() + OakIncrementalIndex.ROW_INDEX_INDEX); + } + + @Override + void setRowIndex(int rowIndex) + { + throw new UnsupportedOperationException(); + } + + @Override + public int copyStringDim(int dimIndex, int[] dst) + { + if (getDimsLength() == 0 || getDimsLength() <= dimIndex) { + return 0; + } + int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + int arrayIndexOffset = oakRBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_INDEX_OFFSET); + int arrayIndex = oakRBuffer.position() + arrayIndexOffset; + int arraySize = oakRBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + if (dst.length < arraySize) { + dst = new int[arraySize]; + } + for (int i = 0; i < arraySize; i++) { + dst[i] = oakRBuffer.getInt(arrayIndex); + arrayIndex += Integer.BYTES; + } + return arraySize; + } + + @Override + public int calcStringDimSize(int dimIndex) + { + if (getDimsLength() == 0 || getDimsLength() <= dimIndex) { + return 0; + } + int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + int arraySize = oakRBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + return arraySize; + } + + /** + * bytesInMemory estimates the size of the serialized IncrementalIndexRow key. + * Each serialized IncrementalRoeIndex contains: + * 1. a timeStamp + * 2. the dims array length + * 3. the rowIndex + * 4. the serialization of each dim + * 5. the array (for dims with capabilities of a String ValueType) + * + * @return long estimated bytesInMemory + */ + @Override + public long estimateBytesInMemory() + { + + long sizeInBytes = Long.BYTES + 2 * Integer.BYTES; + for (int dimIndex = 0; dimIndex < getDimsLength(); dimIndex++) { + sizeInBytes += OakIncrementalIndex.ALLOC_PER_DIM; + int dimType = getDimType(dimIndex); + if (dimType == ValueType.STRING.ordinal()) { + int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + int arraySize = oakRBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + sizeInBytes += (arraySize * Integer.BYTES); + } + } + return sizeInBytes; + } + + + + + /* ---------------- OakRBuffer utils -------------- */ + + private int getDimIndexInBuffer(int dimIndex) + { + if (dimIndex >= getDimsLength()) { + return OakIncrementalIndex.NO_DIM; + } + return oakRBuffer.position() + OakIncrementalIndex.DIMS_INDEX + dimIndex * OakIncrementalIndex.ALLOC_PER_DIM; + } + + private Object getDimValue(int dimIndex) + { + Object dimObject = null; + if (dimIndex >= getDimsLength()) { + return null; + } + int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + int dimType = oakRBuffer.getInt(dimIndexInBuffer); + if (dimType == OakIncrementalIndex.NO_DIM) { + return null; + } else if (dimType == ValueType.DOUBLE.ordinal()) { + dimObject = oakRBuffer.getDouble(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.FLOAT.ordinal()) { + dimObject = oakRBuffer.getFloat(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.LONG.ordinal()) { + dimObject = oakRBuffer.getLong(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.STRING.ordinal()) { + int arrayIndexOffset = oakRBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_INDEX_OFFSET); + int arrayIndex = oakRBuffer.position() + arrayIndexOffset; + int arraySize = oakRBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + int[] array = new int[arraySize]; + for (int i = 0; i < arraySize; i++) { + array[i] = oakRBuffer.getInt(arrayIndex); + arrayIndex += Integer.BYTES; + } + dimObject = array; + } + + return dimObject; + } + + private int getDimType(int dimIndex) + { + if (dimIndex >= getDimsLength()) { + return OakIncrementalIndex.NO_DIM; + } + int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + return oakRBuffer.getInt(dimIndexInBuffer); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java new file mode 100644 index 000000000000..561d3c5179f3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import java.util.List; +import java.nio.ByteBuffer; + +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.incremental.IncrementalIndex.DimensionDesc; +import com.oath.oak.OakSerializer; + +public class OakKeySerializer implements OakSerializer +{ + private List dimensionDescsList; + + public OakKeySerializer(List dimensionDescsList) + { + this.dimensionDescsList = dimensionDescsList; + } + + @Override + public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer buff) + { + long timestamp = incrementalIndexRow.getTimestamp(); + Object[] dims = incrementalIndexRow.getDims(); + int dimsLength = (dims == null ? 0 : dims.length); + int rowIndex = incrementalIndexRow.getRowIndex(); + + // calculating buffer indexes for writing the key data + int buffIndex = buff.position(); // the first byte for writing the key + int timeStampIndex = buffIndex + OakIncrementalIndex.TIME_STAMP_INDEX; // the timestamp index + int dimsLengthIndex = buffIndex + OakIncrementalIndex.DIMS_LENGTH_INDEX; // the dims array length index + int rowIndexIndex = buffIndex + OakIncrementalIndex.ROW_INDEX_INDEX; // the rowIndex index + int dimsIndex = buffIndex + OakIncrementalIndex.DIMS_INDEX; // the dims array index + int dimCapacity = OakIncrementalIndex.ALLOC_PER_DIM; // the number of bytes required + // per dim + int noDim = OakIncrementalIndex.NO_DIM; // for mentioning that + // a certain dim is null + int dimsArraysIndex = dimsIndex + dimCapacity * dimsLength; // the index for + // writing the int arrays + // of dims with a STRING type + int dimsArrayOffset = dimsArraysIndex - buffIndex; // for saving the array position + // in the buffer + int valueTypeOffset = OakIncrementalIndex.VALUE_TYPE_OFFSET; // offset from the dimIndex + int dataOffset = OakIncrementalIndex.DATA_OFFSET; // for non-STRING dims + int arrayIndexOffset = OakIncrementalIndex.ARRAY_INDEX_OFFSET; // for STRING dims + int arrayLengthOffset = OakIncrementalIndex.ARRAY_LENGTH_OFFSET; // for STRING dims + + buff.putLong(timeStampIndex, timestamp); + buff.putInt(dimsLengthIndex, dimsLength); + buff.putInt(rowIndexIndex, rowIndex); + for (int i = 0; i < dimsLength; i++) { + ValueType valueType = OakIncrementalIndex.getDimValueType(i, dimensionDescsList); + if (valueType == null || dims[i] == null) { + buff.putInt(dimsIndex, noDim); + } else { + buff.putInt(dimsIndex + valueTypeOffset, valueType.ordinal()); + switch (valueType) { + case FLOAT: + buff.putFloat(dimsIndex + dataOffset, (Float) dims[i]); + break; + case DOUBLE: + buff.putDouble(dimsIndex + dataOffset, (Double) dims[i]); + break; + case LONG: + buff.putLong(dimsIndex + dataOffset, (Long) dims[i]); + break; + case STRING: + int[] arr = (int[]) dims[i]; + buff.putInt(dimsIndex + arrayIndexOffset, dimsArrayOffset); + buff.putInt(dimsIndex + arrayLengthOffset, arr.length); + for (int arrIndex = 0; arrIndex < arr.length; arrIndex++) { + buff.putInt(dimsArraysIndex + arrIndex * Integer.BYTES, arr[arrIndex]); + } + dimsArraysIndex += (arr.length * Integer.BYTES); + dimsArrayOffset += (arr.length * Integer.BYTES); + break; + default: + buff.putInt(dimsIndex, noDim); + } + } + + dimsIndex += dimCapacity; + } + + } + + @Override + public IncrementalIndexRow deserialize(ByteBuffer serializedKey) + { + long timeStamp = OakIncrementalIndex.getTimestamp(serializedKey); + int dimsLength = OakIncrementalIndex.getDimsLength(serializedKey); + int rowIndex = OakIncrementalIndex.getRowIndex(serializedKey); + Object[] dims = new Object[dimsLength]; + for (int dimIndex = 0; dimIndex < dimsLength; dimIndex++) { + Object dim = OakIncrementalIndex.getDimValue(serializedKey, dimIndex, dimsLength); + dims[dimIndex] = dim; + } + return new IncrementalIndexRow(timeStamp, dims, dimensionDescsList, rowIndex); + } + + @Override + public int calculateSize(IncrementalIndexRow incrementalIndexRow) + { + Object[] dims = incrementalIndexRow.getDims(); + if (dims == null) { + return Long.BYTES + 2 * Integer.BYTES; + } + + // When the dimensionDesc's capabilities are of type ValueType.STRING, + // the object in timeAndDims.dims is of type int[]. + // In this case, we need to know the array size before allocating the ByteBuffer. + int sumOfArrayLengths = 0; + for (int i = 0; i < dims.length; i++) { + Object dim = dims[i]; + if (dim == null) { + continue; + } + if (OakIncrementalIndex.getDimValueType(i, dimensionDescsList) == ValueType.STRING) { + sumOfArrayLengths += ((int[]) dim).length; + } + } + + // The ByteBuffer will contain: + // 1. the timeStamp + // 2. dims.length + // 3. rowIndex (used for Plain mode only) + // 4. the serialization of each dim + // 5. the array (for dims with capabilities of a String ValueType) + int dimCapacity = OakIncrementalIndex.ALLOC_PER_DIM; + int allocSize = Long.BYTES + 2 * Integer.BYTES + dimCapacity * dims.length + Integer.BYTES * sumOfArrayLengths; + return allocSize; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java new file mode 100644 index 000000000000..95e6e5df7542 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import org.apache.druid.segment.DimensionIndexer; +import com.oath.oak.OakComparator; + +import java.nio.ByteBuffer; +import java.util.List; + +public class OakKeysComparator implements OakComparator +{ + + private List dimensionDescsList; + private boolean rollup; + + public OakKeysComparator(List dimensionDescsList, boolean rollup) + { + this.dimensionDescsList = dimensionDescsList; + this.rollup = rollup; + } + + @Override + public int compareKeys(IncrementalIndexRow lhs, IncrementalIndexRow rhs) + { + int retVal = Longs.compare(lhs.getTimestamp(), rhs.getTimestamp()); + int lhsDimsLength = lhs.getDimsLength(); + int rhsDimsLength = rhs.getDimsLength(); + int numComparisons = Math.min(lhsDimsLength, rhsDimsLength); + + int index = 0; + while (retVal == 0 && index < numComparisons) { + final Object lhsIdxs = lhs.getDim(index); + final Object rhsIdxs = rhs.getDim(index); + + if (lhsIdxs == null) { + if (rhsIdxs == null) { + ++index; + continue; + } + return -1; + } + + if (rhsIdxs == null) { + return 1; + } + + final DimensionIndexer indexer = dimensionDescsList.get(index).getIndexer(); + retVal = indexer.compareUnsortedEncodedKeyComponents(lhsIdxs, rhsIdxs); + ++index; + } + + if (retVal == 0) { + int lengthDiff = Ints.compare(lhsDimsLength, rhsDimsLength); + if (lengthDiff == 0) { + if (rollup) { + return 0; + } + return lhs.getRowIndex() - rhs.getRowIndex(); + } + IncrementalIndexRow largerDims = lengthDiff > 0 ? lhs : rhs; + if (IncrementalIndex.allNull(largerDims, numComparisons)) { + if (rollup) { + return 0; + } + return lhs.getRowIndex() - rhs.getRowIndex(); + } + return lengthDiff; + } + + return retVal; + } + + @Override + public int compareSerializedKeys(ByteBuffer lhs, ByteBuffer rhs) + { + int retVal = Longs.compare(OakIncrementalIndex.getTimestamp(lhs), OakIncrementalIndex.getTimestamp(rhs)); + int lhsDimsLength = OakIncrementalIndex.getDimsLength(lhs); + int rhsDimsLength = OakIncrementalIndex.getDimsLength(rhs); + int numComparisons = Math.min(lhsDimsLength, rhsDimsLength); + + int dimIndex = 0; + while (retVal == 0 && dimIndex < numComparisons) { + int lhsType = lhs.getInt(OakIncrementalIndex.getDimIndexInBuffer(lhs, lhsDimsLength, dimIndex)); + int rhsType = rhs.getInt(OakIncrementalIndex.getDimIndexInBuffer(rhs, rhsDimsLength, dimIndex)); + + if (lhsType == OakIncrementalIndex.NO_DIM) { + if (rhsType == OakIncrementalIndex.NO_DIM) { + ++dimIndex; + continue; + } + return -1; + } + + if (rhsType == OakIncrementalIndex.NO_DIM) { + return 1; + } + + final DimensionIndexer indexer = dimensionDescsList.get(dimIndex).getIndexer(); + Object lhsObject = OakIncrementalIndex.getDimValue(lhs, dimIndex); + Object rhsObject = OakIncrementalIndex.getDimValue(rhs, dimIndex); + retVal = indexer.compareUnsortedEncodedKeyComponents(lhsObject, rhsObject); + ++dimIndex; + } + + if (retVal == 0) { + int lengthDiff = Ints.compare(lhsDimsLength, rhsDimsLength); + if (lengthDiff == 0) { + if (rollup) { + return 0; + } + return OakIncrementalIndex.getRowIndex(lhs) - OakIncrementalIndex.getRowIndex(rhs); + } + ByteBuffer largerDims = lengthDiff > 0 ? lhs : rhs; + if (OakIncrementalIndex.checkDimsAllNull(largerDims, numComparisons)) { + if (rollup) { + return 0; + } + return OakIncrementalIndex.getRowIndex(lhs) - OakIncrementalIndex.getRowIndex(rhs); + } + return lengthDiff; + } + return retVal; + } + + @Override + public int compareSerializedKeyAndKey(ByteBuffer lhs, IncrementalIndexRow rhs) + { + int retVal = Longs.compare(OakIncrementalIndex.getTimestamp(lhs), rhs.getTimestamp()); + int lhsDimsLength = OakIncrementalIndex.getDimsLength(lhs); + int rhsDimsLength = rhs.getDimsLength(); + int numComparisons = Math.min(lhsDimsLength, rhsDimsLength); + + int index = 0; + while (retVal == 0 && index < numComparisons) { + final Object lhsIdxs = OakIncrementalIndex.getDimValue(lhs, index); + final Object rhsIdxs = rhs.getDim(index); + + if (lhsIdxs == null) { + if (rhsIdxs == null) { + ++index; + continue; + } + return -1; + } + + if (rhsIdxs == null) { + return 1; + } + + final DimensionIndexer indexer = dimensionDescsList.get(index).getIndexer(); + retVal = indexer.compareUnsortedEncodedKeyComponents(lhsIdxs, rhsIdxs); + ++index; + } + + if (retVal == 0) { + int lengthDiff = Ints.compare(lhsDimsLength, rhsDimsLength); + if (lengthDiff == 0) { + if (rollup) { + return 0; + } + return OakIncrementalIndex.getRowIndex(lhs) - rhs.getRowIndex(); + } + + if (lengthDiff > 0) { + if (OakIncrementalIndex.checkDimsAllNull(lhs, numComparisons)) { + if (rollup) { + return 0; + } + return OakIncrementalIndex.getRowIndex(lhs) - rhs.getRowIndex(); + } + } else { + if (OakIncrementalIndex.allNull(rhs, numComparisons)) { + if (rollup) { + return 0; + } + return OakIncrementalIndex.getRowIndex(lhs) - rhs.getRowIndex(); + } + } + return lengthDiff; + } + + return retVal; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakValueSerializer.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakValueSerializer.java new file mode 100644 index 000000000000..2a28842cb24c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakValueSerializer.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.incremental; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.Row; + +import java.nio.ByteBuffer; +import com.oath.oak.OakSerializer; + +public class OakValueSerializer implements OakSerializer +{ + + private OffheapAggsManager aggsManager; + private boolean reportParseExceptions; + private ThreadLocal rowContainer; + + public OakValueSerializer( + OffheapAggsManager aggsManager, + boolean reportParseExceptions, + ThreadLocal rowContainer + ) + { + this.aggsManager = aggsManager; + this.reportParseExceptions = reportParseExceptions; + this.rowContainer = rowContainer; + } + + @Override + public void serialize(Row value, ByteBuffer byteBuffer) + { + aggsManager.initValue(byteBuffer, reportParseExceptions, (InputRow) value, rowContainer); + } + + @Override + public Row deserialize(ByteBuffer serializedValue) + { + throw new UnsupportedOperationException(); // cannot be deserialized without the IncrementalIndexRow + } + + @Override + public int calculateSize(Row row) + { + return aggsManager.aggsTotalSize; + } + +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OffheapAggsManager.java b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapAggsManager.java new file mode 100644 index 000000000000..b69199e7a35e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapAggsManager.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import com.google.common.base.Supplier; +import com.google.common.collect.Maps; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.ParseException; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.BufferAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; + +import java.util.Map; + +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; + +import java.nio.ByteBuffer; + +public class OffheapAggsManager extends AggsManager +{ + private static final Logger log = new Logger(OffheapAggsManager.class); + + public volatile Map selectors; + public volatile int[] aggOffsetInBuffer; + public volatile int aggsTotalSize; + NonBlockingPool bufferPool; + + /* basic constractor */ + OffheapAggsManager( + final IncrementalIndexSchema incrementalIndexSchema, + final boolean deserializeComplexMetrics, + final boolean concurrentEventAdd, + Supplier rowSupplier, + Map columnCapabilities, + NonBlockingPool bufferPool, + IncrementalIndex incrementalIndex + ) + { + super(incrementalIndexSchema, deserializeComplexMetrics, + concurrentEventAdd, rowSupplier, columnCapabilities, incrementalIndex); + this.bufferPool = bufferPool; + } + + @Override + protected BufferAggregator[] initAggs( + AggregatorFactory[] metrics, + Supplier rowSupplier, + boolean deserializeComplexMetrics, + boolean concurrentEventAdd + ) + { + selectors = Maps.newHashMap(); + aggOffsetInBuffer = new int[metrics.length]; + + for (int i = 0; i < metrics.length; i++) { + AggregatorFactory agg = metrics[i]; + + ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactory( + agg, + rowSupplier, + deserializeComplexMetrics + ); + + selectors.put( + agg.getName(), + new OnheapIncrementalIndex.CachingColumnSelectorFactory(columnSelectorFactory, concurrentEventAdd) + ); + + if (i == 0) { + aggOffsetInBuffer[i] = 0; + } else { + aggOffsetInBuffer[i] = aggOffsetInBuffer[i - 1] + metrics[i - 1].getMaxIntermediateSizeWithNulls(); + } + } + aggsTotalSize += aggOffsetInBuffer[metrics.length - 1] + metrics[metrics.length - 1].getMaxIntermediateSizeWithNulls(); + + return new BufferAggregator[metrics.length]; + } + + public void clearSelectors() + { + if (selectors != null) { + selectors.clear(); + } + } + + public void initValue(ByteBuffer byteBuffer, + boolean reportParseExceptions, + InputRow row, + ThreadLocal rowContainer) + { + if (metrics.length > 0 && aggs[aggs.length - 1] == null) { + // note: creation of Aggregators is done lazily when at least one row from input is available + // so that FilteredAggregators could be initialized correctly. + rowContainer.set(row); + for (int i = 0; i < metrics.length; i++) { + final AggregatorFactory agg = metrics[i]; + aggLocks[i].lock(); + if (aggs[i] == null) { + aggs[i] = agg.factorizeBuffered(selectors.get(agg.getName())); + } + aggLocks[i].unlock(); + } + rowContainer.set(null); + } + + for (int i = 0; i < metrics.length; i++) { + aggs[i].init(byteBuffer, aggOffsetInBuffer[i]); + } + + aggregate(reportParseExceptions, row, rowContainer, byteBuffer); + } + + public void aggregate( + boolean reportParseExceptions, + InputRow row, + ThreadLocal rowContainer, + ByteBuffer aggBuffer + ) + { + rowContainer.set(row); + + for (int i = 0; i < metrics.length; i++) { + final BufferAggregator agg = aggs[i]; + + synchronized (agg) { + try { + agg.aggregate(aggBuffer, aggBuffer.position() + aggOffsetInBuffer[i]); + } + catch (ParseException e) { + // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. + if (reportParseExceptions) { + throw new ParseException(e, "Encountered parse error for aggregator[%s]", metrics[i].getName()); + } else { + log.debug(e, "Encountered parse error, skipping aggregator[%s].", metrics[i].getName()); + } + } + } + } + rowContainer.set(null); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java index 3f7f2fedd7bc..348bc3dc08f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OffheapIncrementalIndex.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.incremental; import com.google.common.base.Supplier; -import com.google.common.collect.Maps; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.data.input.InputRow; @@ -32,42 +31,28 @@ import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.BufferAggregator; -import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; /** */ -public class OffheapIncrementalIndex extends IncrementalIndex +public class OffheapIncrementalIndex extends ExternalDataIncrementalIndex { private static final Logger log = new Logger(OffheapIncrementalIndex.class); - private final NonBlockingPool bufferPool; - private final List> aggBuffers = new ArrayList<>(); private final List indexAndOffsets = new ArrayList<>(); - private final FactsHolder facts; - private final AtomicInteger indexIncrement = new AtomicInteger(0); - protected final int maxRowCount; - - private volatile Map selectors; - - //given a ByteBuffer and an offset where all aggregates for a row are stored - //offset + aggOffsetInBuffer[i] would give position in ByteBuffer where ith aggregate - //is stored - private volatile int[] aggOffsetInBuffer; - private volatile int aggsTotalSize; - - private String outOfRowsReason = null; + protected OffheapAggsManager aggsManager; OffheapIncrementalIndex( IncrementalIndexSchema incrementalIndexSchema, @@ -79,69 +64,34 @@ public class OffheapIncrementalIndex extends IncrementalIndex NonBlockingPool bufferPool ) { - super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); - this.maxRowCount = maxRowCount; - this.bufferPool = bufferPool; - - this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) - : new PlainFactsHolder(sortFacts, dimsComparator()); + super(incrementalIndexSchema, reportParseExceptions, sortFacts, maxRowCount); + this.aggsManager = new OffheapAggsManager(incrementalIndexSchema, deserializeComplexMetrics, + concurrentEventAdd, rowSupplier, + columnCapabilities, bufferPool, this); //check that stupid pool gives buffers that can hold at least one row's aggregators - ResourceHolder bb = bufferPool.take(); - if (bb.get().capacity() < aggsTotalSize) { + ResourceHolder bb = aggsManager.bufferPool.take(); + if (bb.get().capacity() < this.aggsManager.aggsTotalSize) { bb.close(); - throw new IAE("bufferPool buffers capacity must be >= [%s]", aggsTotalSize); + throw new IAE("bufferPool buffers capacity must be >= [%s]", this.aggsManager.aggsTotalSize); } aggBuffers.add(bb); } @Override - public FactsHolder getFacts() + protected FactsHolder getFacts() { return facts; } @Override - protected BufferAggregator[] initAggs( - final AggregatorFactory[] metrics, - final Supplier rowSupplier, - final boolean deserializeComplexMetrics, - final boolean concurrentEventAdd - ) + public AggregatorFactory[] getMetricAggs() { - selectors = Maps.newHashMap(); - aggOffsetInBuffer = new int[metrics.length]; - - for (int i = 0; i < metrics.length; i++) { - AggregatorFactory agg = metrics[i]; - - ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactory( - agg, - rowSupplier, - deserializeComplexMetrics - ); - - selectors.put( - agg.getName(), - new OnheapIncrementalIndex.CachingColumnSelectorFactory(columnSelectorFactory, concurrentEventAdd) - ); - - if (i == 0) { - aggOffsetInBuffer[i] = 0; - } else { - aggOffsetInBuffer[i] = aggOffsetInBuffer[i - 1] + metrics[i - 1].getMaxIntermediateSizeWithNulls(); - } - } - - aggsTotalSize = aggOffsetInBuffer[metrics.length - 1] + metrics[metrics.length - 1].getMaxIntermediateSizeWithNulls(); - - return new BufferAggregator[metrics.length]; + return aggsManager.getMetricAggs(); } @Override protected AddToFactsResult addToFacts( - AggregatorFactory[] metrics, - boolean deserializeComplexMetrics, boolean reportParseExceptions, InputRow row, AtomicInteger numEntries, @@ -164,13 +114,13 @@ protected AddToFactsResult addToFacts( bufferOffset = indexAndOffset[1]; aggBuffer = aggBuffers.get(bufferIndex).get(); } else { - if (metrics.length > 0 && getAggs()[0] == null) { + if (aggsManager.metrics.length > 0 && aggsManager.getAggs()[0] == null) { // note: creation of Aggregators is done lazily when at least one row from input is available // so that FilteredAggregators could be initialized correctly. rowContainer.set(row); - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; - getAggs()[i] = agg.factorizeBuffered(selectors.get(agg.getName())); + for (int i = 0; i < aggsManager.metrics.length; i++) { + final AggregatorFactory agg = aggsManager.metrics[i]; + aggsManager.getAggs()[i] = agg.factorizeBuffered(aggsManager.selectors.get(agg.getName())); } rowContainer.set(null); } @@ -178,27 +128,27 @@ protected AddToFactsResult addToFacts( bufferIndex = aggBuffers.size() - 1; ByteBuffer lastBuffer = aggBuffers.isEmpty() ? null : aggBuffers.get(aggBuffers.size() - 1).get(); int[] lastAggregatorsIndexAndOffset = indexAndOffsets.isEmpty() - ? null - : indexAndOffsets.get(indexAndOffsets.size() - 1); + ? null + : indexAndOffsets.get(indexAndOffsets.size() - 1); if (lastAggregatorsIndexAndOffset != null && lastAggregatorsIndexAndOffset[0] != bufferIndex) { throw new ISE("last row's aggregate's buffer and last buffer index must be same"); } - bufferOffset = aggsTotalSize + (lastAggregatorsIndexAndOffset != null ? lastAggregatorsIndexAndOffset[1] : 0); + bufferOffset = aggsManager.aggsTotalSize + (lastAggregatorsIndexAndOffset != null ? lastAggregatorsIndexAndOffset[1] : 0); if (lastBuffer != null && - lastBuffer.capacity() - bufferOffset >= aggsTotalSize) { + lastBuffer.capacity() - bufferOffset >= aggsManager.aggsTotalSize) { aggBuffer = lastBuffer; } else { - ResourceHolder bb = bufferPool.take(); + ResourceHolder bb = aggsManager.bufferPool.take(); aggBuffers.add(bb); bufferIndex = aggBuffers.size() - 1; bufferOffset = 0; aggBuffer = bb.get(); } - for (int i = 0; i < metrics.length; i++) { - getAggs()[i].init(aggBuffer, bufferOffset + aggOffsetInBuffer[i]); + for (int i = 0; i < aggsManager.metrics.length; i++) { + aggsManager.getAggs()[i].init(aggBuffer, bufferOffset + aggsManager.aggOffsetInBuffer[i]); } // Last ditch sanity checks @@ -222,12 +172,12 @@ protected AddToFactsResult addToFacts( rowContainer.set(row); - for (int i = 0; i < metrics.length; i++) { - final BufferAggregator agg = getAggs()[i]; + for (int i = 0; i < aggsManager.metrics.length; i++) { + final BufferAggregator agg = aggsManager.getAggs()[i]; synchronized (agg) { try { - agg.aggregate(aggBuffer, bufferOffset + aggOffsetInBuffer[i]); + agg.aggregate(aggBuffer, bufferOffset + aggsManager.aggOffsetInBuffer[i]); } catch (ParseException e) { // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. @@ -266,62 +216,63 @@ public String getOutOfRowsReason() } @Override - protected BufferAggregator[] getAggsForRow(int rowOffset) + protected BufferAggregator[] getAggsForRow(IncrementalIndexRow incrementalIndexRow) { - return getAggs(); + return aggsManager.getAggs(); } @Override - protected Object getAggVal(BufferAggregator agg, int rowOffset, int aggPosition) + protected Object getAggVal(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - int[] indexAndOffset = indexAndOffsets.get(rowOffset); - ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); - return agg.get(bb, indexAndOffset[1] + aggOffsetInBuffer[aggPosition]); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); + ByteBuffer aggBuffer = aggBuffers.get(indexAndOffset[0]).get(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.get(aggBuffer, indexAndOffset[1] + aggsManager.aggOffsetInBuffer[aggIndex]); } @Override - public float getMetricFloatValue(int rowOffset, int aggOffset) + public float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); - ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); - return agg.getFloat(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); + ByteBuffer aggBuffer = aggBuffers.get(indexAndOffset[0]).get(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.getFloat(aggBuffer, indexAndOffset[1] + aggsManager.aggOffsetInBuffer[aggIndex]); } @Override - public long getMetricLongValue(int rowOffset, int aggOffset) + public long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); - ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); - return agg.getLong(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); + ByteBuffer aggBuffer = aggBuffers.get(indexAndOffset[0]).get(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.getLong(aggBuffer, indexAndOffset[1] + aggsManager.aggOffsetInBuffer[aggIndex]); } @Override - public Object getMetricObjectValue(int rowOffset, int aggOffset) + public Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); - ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); - return agg.get(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); + ByteBuffer aggBuffer = aggBuffers.get(indexAndOffset[0]).get(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.get(aggBuffer, indexAndOffset[1] + aggsManager.aggOffsetInBuffer[aggIndex]); } @Override - public double getMetricDoubleValue(int rowOffset, int aggOffset) + public double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); - ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); - return agg.getDouble(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); + ByteBuffer aggBuffer = aggBuffers.get(indexAndOffset[0]).get(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.getDouble(aggBuffer, indexAndOffset[1] + aggsManager.aggOffsetInBuffer[aggIndex]); } @Override - public boolean isNull(int rowOffset, int aggOffset) + public boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); - ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); - return agg.isNull(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); + ByteBuffer aggBuffer = aggBuffers.get(indexAndOffset[0]).get(); + BufferAggregator agg = aggsManager.getAggs()[aggIndex]; + return agg.isNull(aggBuffer, indexAndOffset[1] + aggsManager.aggOffsetInBuffer[aggIndex]); } /** @@ -334,9 +285,7 @@ public void close() facts.clear(); indexAndOffsets.clear(); - if (selectors != null) { - selectors.clear(); - } + aggsManager.clearSelectors(); Closer c = Closer.create(); aggBuffers.forEach(c::register); @@ -348,4 +297,29 @@ public void close() } aggBuffers.clear(); } + + @Nullable + @Override + public String getMetricType(String metric) + { + return aggsManager.getMetricType(metric); + } + + @Override + public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) + { + return aggsManager.makeMetricColumnValueSelector(metric, currEntry); + } + + @Override + public List getMetricNames() + { + return aggsManager.getMetricNames(); + } + + @Override + protected String getMetricName(int metricIndex) + { + return aggsManager.metrics[metricIndex].getName(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OnheapAggsManager.java b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapAggsManager.java new file mode 100644 index 000000000000..92fda5ca39a2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OnheapAggsManager.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import com.google.common.base.Supplier; +import com.google.common.base.Throwables; +import com.google.common.collect.Maps; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.io.Closer; +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.AggregatorFactory; +import org.apache.druid.segment.ColumnSelectorFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; + +public class OnheapAggsManager extends AggsManager +{ + private static final Logger log = new Logger(OnheapAggsManager.class); + + private volatile Map selectors; + private final ConcurrentHashMap aggregators; + + /* basic constractor */ + OnheapAggsManager( + final IncrementalIndexSchema incrementalIndexSchema, + final boolean deserializeComplexMetrics, + final boolean concurrentEventAdd, + Supplier rowSupplier, + Map columnCapabilities, + IncrementalIndex incrementalIndex + ) + { + super(incrementalIndexSchema, deserializeComplexMetrics, + concurrentEventAdd, rowSupplier, columnCapabilities, incrementalIndex); + this.aggregators = new ConcurrentHashMap<>(); + } + + public void closeAggregators() + { + Closer closer = Closer.create(); + for (Aggregator[] aggs : aggregators.values()) { + for (Aggregator agg : aggs) { + closer.register(agg); + } + } + + try { + closer.close(); + } + catch (IOException e) { + Throwables.propagate(e); + } + } + + public void clearAggregators() + { + aggregators.clear(); + } + + public void clearSelectors() + { + if (selectors != null) { + selectors.clear(); + } + } + + public Aggregator[] concurrentGet(int offset) + { + // All get operations should be fine + return aggregators.get(offset); + } + + public void concurrentSet(int offset, Aggregator[] value) + { + aggregators.put(offset, value); + } + + public void concurrentRemove(int offset) + { + aggregators.remove(offset); + } + + public Aggregator[] concurrentPutIfAbsent(int offset, Aggregator[] value) + { + return aggregators.putIfAbsent(offset, value); + } + + @Override + public Aggregator[] initAggs( + AggregatorFactory[] metrics, + Supplier rowSupplier, + boolean deserializeComplexMetrics, + boolean concurrentEventAdd + ) + { + selectors = Maps.newHashMap(); + for (AggregatorFactory agg : metrics) { + selectors.put( + agg.getName(), + new OnheapIncrementalIndex.CachingColumnSelectorFactory( + makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics), + concurrentEventAdd + ) + ); + } + + return new Aggregator[metrics.length]; + } + + public void factorizeAggs( + AggregatorFactory[] metrics, + Aggregator[] aggs, + ThreadLocal rowContainer, + InputRow row + ) + { + rowContainer.set(row); + for (int i = 0; i < metrics.length; i++) { + final AggregatorFactory agg = metrics[i]; + aggs[i] = agg.factorize(selectors.get(agg.getName())); + } + rowContainer.set(null); + } + + public List doAggregate( + AggregatorFactory[] metrics, + Aggregator[] aggs, + ThreadLocal rowContainer, + InputRow row + ) + { + List parseExceptionMessages = new ArrayList<>(); + rowContainer.set(row); + + for (int i = 0; i < aggs.length; i++) { + final Aggregator agg = aggs[i]; + synchronized (agg) { + try { + agg.aggregate(); + } + catch (ParseException e) { + // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. + log.debug(e, "Encountered parse error, skipping aggregator[%s].", metrics[i].getName()); + parseExceptionMessages.add(e.getMessage()); + } + } + } + + rowContainer.set(null); + return parseExceptionMessages; + } + +} 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 02f0a37cee56..d284f06aac64 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 @@ -20,13 +20,8 @@ package org.apache.druid.segment.incremental; import com.google.common.base.Supplier; -import com.google.common.base.Throwables; -import com.google.common.collect.Maps; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.io.Closer; -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.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; @@ -36,8 +31,6 @@ import org.apache.druid.segment.column.ColumnCapabilities; import javax.annotation.Nullable; -import java.io.IOException; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -48,38 +41,32 @@ /** */ -public class OnheapIncrementalIndex extends IncrementalIndex +public class OnheapIncrementalIndex extends ExternalDataIncrementalIndex { - private static final Logger log = new Logger(OnheapIncrementalIndex.class); /** * overhead per {@link ConcurrentHashMap.Node} or {@link java.util.concurrent.ConcurrentSkipListMap.Node} object */ private static final int ROUGH_OVERHEAD_PER_MAP_ENTRY = Long.BYTES * 5 + Integer.BYTES; - private final ConcurrentHashMap aggregators = new ConcurrentHashMap<>(); - private final FactsHolder facts; - private final AtomicInteger indexIncrement = new AtomicInteger(0); + + protected final AtomicInteger indexIncrement = new AtomicInteger(0); private final long maxBytesPerRowForAggregators; - protected final int maxRowCount; protected final long maxBytesInMemory; - private volatile Map selectors; - - private String outOfRowsReason = null; + protected OnheapAggsManager aggsManager; OnheapIncrementalIndex( - IncrementalIndexSchema incrementalIndexSchema, - boolean deserializeComplexMetrics, - boolean reportParseExceptions, - boolean concurrentEventAdd, - boolean sortFacts, - int maxRowCount, - long maxBytesInMemory + IncrementalIndexSchema incrementalIndexSchema, + boolean deserializeComplexMetrics, + boolean reportParseExceptions, + boolean concurrentEventAdd, + boolean sortFacts, + int maxRowCount, + long maxBytesInMemory ) { - super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); - this.maxRowCount = maxRowCount; + super(incrementalIndexSchema, reportParseExceptions, sortFacts, maxRowCount); + this.aggsManager = new OnheapAggsManager(incrementalIndexSchema, deserializeComplexMetrics, + concurrentEventAdd, rowSupplier, columnCapabilities, this); this.maxBytesInMemory = maxBytesInMemory == 0 ? Long.MAX_VALUE : maxBytesInMemory; - this.facts = incrementalIndexSchema.isRollup() ? new RollupFactsHolder(sortFacts, dimsComparator(), getDimensions()) - : new PlainFactsHolder(sortFacts, dimsComparator()); maxBytesPerRowForAggregators = getMaxBytesPerRowForAggregators(incrementalIndexSchema); } @@ -112,45 +99,27 @@ private static long getMaxBytesPerRowForAggregators(IncrementalIndexSchema incre } @Override - public FactsHolder getFacts() + protected FactsHolder getFacts() { return facts; } @Override - protected Aggregator[] initAggs( - final AggregatorFactory[] metrics, - final Supplier rowSupplier, - final boolean deserializeComplexMetrics, - final boolean concurrentEventAdd - ) + public AggregatorFactory[] getMetricAggs() { - selectors = Maps.newHashMap(); - for (AggregatorFactory agg : metrics) { - selectors.put( - agg.getName(), - new CachingColumnSelectorFactory( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics), - concurrentEventAdd - ) - ); - } - - return new Aggregator[metrics.length]; + return aggsManager.getMetricAggs(); } @Override protected AddToFactsResult addToFacts( - AggregatorFactory[] metrics, - boolean deserializeComplexMetrics, - boolean reportParseExceptions, - InputRow row, - AtomicInteger numEntries, - AtomicLong sizeInBytes, - IncrementalIndexRow key, - ThreadLocal rowContainer, - Supplier rowSupplier, - boolean skipMaxRowsInMemoryCheck + boolean reportParseExceptions, + InputRow row, + AtomicInteger numEntries, + AtomicLong sizeInBytes, + IncrementalIndexRow key, + ThreadLocal rowContainer, + Supplier rowSupplier, + boolean skipMaxRowsInMemoryCheck ) throws IndexSizeExceededException { List parseExceptionMessages; @@ -159,15 +128,15 @@ protected AddToFactsResult addToFacts( Aggregator[] aggs; if (IncrementalIndexRow.EMPTY_ROW_INDEX != priorIndex) { - aggs = concurrentGet(priorIndex); - parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row); + aggs = aggsManager.concurrentGet(priorIndex); + parseExceptionMessages = aggsManager.doAggregate(aggsManager.metrics, aggs, rowContainer, row); } else { - aggs = new Aggregator[metrics.length]; - factorizeAggs(metrics, aggs, rowContainer, row); - parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row); + aggs = new Aggregator[aggsManager.metrics.length]; + aggsManager.factorizeAggs(aggsManager.metrics, aggs, rowContainer, row); + parseExceptionMessages = aggsManager.doAggregate(aggsManager.metrics, aggs, rowContainer, row); final int rowIndex = indexIncrement.getAndIncrement(); - concurrentSet(rowIndex, aggs); + aggsManager.concurrentSet(rowIndex, aggs); // Last ditch sanity checks if ((numEntries.get() >= maxRowCount || sizeInBytes.get() >= maxBytesInMemory) @@ -186,10 +155,10 @@ protected AddToFactsResult addToFacts( sizeInBytes.addAndGet(estimatedRowSize); } else { // We lost a race - aggs = concurrentGet(prev); - parseExceptionMessages = doAggregate(metrics, aggs, rowContainer, row); + aggs = aggsManager.concurrentGet(prev); + parseExceptionMessages = aggsManager.doAggregate(aggsManager.metrics, aggs, rowContainer, row); // Free up the misfire - concurrentRemove(rowIndex); + aggsManager.concurrentRemove(rowIndex); // This is expected to occur ~80% of the time in the worst scenarios } } @@ -221,82 +190,6 @@ public int getLastRowIndex() return indexIncrement.get() - 1; } - private void factorizeAggs( - AggregatorFactory[] metrics, - Aggregator[] aggs, - ThreadLocal rowContainer, - InputRow row - ) - { - rowContainer.set(row); - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; - aggs[i] = agg.factorize(selectors.get(agg.getName())); - } - rowContainer.set(null); - } - - private List doAggregate( - AggregatorFactory[] metrics, - Aggregator[] aggs, - ThreadLocal rowContainer, - InputRow row - ) - { - List parseExceptionMessages = new ArrayList<>(); - rowContainer.set(row); - - for (int i = 0; i < aggs.length; i++) { - final Aggregator agg = aggs[i]; - synchronized (agg) { - try { - agg.aggregate(); - } - catch (ParseException e) { - // "aggregate" can throw ParseExceptions if a selector expects something but gets something else. - log.debug(e, "Encountered parse error, skipping aggregator[%s].", metrics[i].getName()); - parseExceptionMessages.add(e.getMessage()); - } - } - } - - rowContainer.set(null); - return parseExceptionMessages; - } - - private void closeAggregators() - { - Closer closer = Closer.create(); - for (Aggregator[] aggs : aggregators.values()) { - for (Aggregator agg : aggs) { - closer.register(agg); - } - } - - try { - closer.close(); - } - catch (IOException e) { - Throwables.propagate(e); - } - } - - protected Aggregator[] concurrentGet(int offset) - { - // All get operations should be fine - return aggregators.get(offset); - } - - protected void concurrentSet(int offset, Aggregator[] value) - { - aggregators.put(offset, value); - } - - protected void concurrentRemove(int offset) - { - aggregators.remove(offset); - } - @Override public boolean canAppendRow() { @@ -328,45 +221,52 @@ public String getOutOfRowsReason() } @Override - protected Aggregator[] getAggsForRow(int rowOffset) + protected Aggregator[] getAggsForRow(IncrementalIndexRow incrementalIndexRow) { - return concurrentGet(rowOffset); + int rowIndex = incrementalIndexRow.getRowIndex(); + return aggsManager.concurrentGet(rowIndex); } @Override - protected Object getAggVal(Aggregator agg, int rowOffset, int aggPosition) + protected Object getAggVal(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - return agg.get(); + int rowIndex = incrementalIndexRow.getRowIndex(); + return aggsManager.concurrentGet(rowIndex)[aggIndex].get(); } @Override - public float getMetricFloatValue(int rowOffset, int aggOffset) + public float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - return concurrentGet(rowOffset)[aggOffset].getFloat(); + int rowIndex = incrementalIndexRow.getRowIndex(); + return aggsManager.concurrentGet(rowIndex)[aggIndex].getFloat(); } @Override - public long getMetricLongValue(int rowOffset, int aggOffset) + public long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - return concurrentGet(rowOffset)[aggOffset].getLong(); + int rowIndex = incrementalIndexRow.getRowIndex(); + return aggsManager.concurrentGet(rowIndex)[aggIndex].getLong(); } @Override - public Object getMetricObjectValue(int rowOffset, int aggOffset) + public Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - return concurrentGet(rowOffset)[aggOffset].get(); + int rowIndex = incrementalIndexRow.getRowIndex(); + return aggsManager.concurrentGet(rowIndex)[aggIndex].get(); } @Override - protected double getMetricDoubleValue(int rowOffset, int aggOffset) + public double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - return concurrentGet(rowOffset)[aggOffset].getDouble(); + int rowIndex = incrementalIndexRow.getRowIndex(); + return aggsManager.concurrentGet(rowIndex)[aggIndex].getDouble(); } @Override - public boolean isNull(int rowOffset, int aggOffset) + public boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - return concurrentGet(rowOffset)[aggOffset].isNull(); + int rowIndex = incrementalIndexRow.getRowIndex(); + return aggsManager.concurrentGet(rowIndex)[aggIndex].isNull(); } /** @@ -377,12 +277,10 @@ public boolean isNull(int rowOffset, int aggOffset) public void close() { super.close(); - closeAggregators(); - aggregators.clear(); + aggsManager.closeAggregators(); + aggsManager.clearAggregators(); facts.clear(); - if (selectors != null) { - selectors.clear(); - } + aggsManager.clearSelectors(); } /** @@ -430,4 +328,29 @@ public ColumnCapabilities getColumnCapabilities(String columnName) } } + @Nullable + @Override + public String getMetricType(String metric) + { + return aggsManager.getMetricType(metric); + } + + @Override + public ColumnValueSelector makeMetricColumnValueSelector(String metric, IncrementalIndexRowHolder currEntry) + { + return aggsManager.makeMetricColumnValueSelector(metric, currEntry); + } + + @Override + public List getMetricNames() + { + return aggsManager.getMetricNames(); + } + + @Override + protected String getMetricName(int metricIndex) + { + return aggsManager.metrics[metricIndex].getName(); + } + } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexAdapterTest.java index 11182c7140f9..e14a7ccd6fbf 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexAdapterTest.java @@ -100,12 +100,12 @@ public void testGetRowsIterableNoRollup() throws Exception ArrayList dim1Vals = new ArrayList<>(); - for (IncrementalIndexRow row : toPersist1.getFacts().keySet()) { - dim1Vals.add(((int[]) row.getDims()[0])[0]); + for (IncrementalIndexRow row : (Iterable) toPersist1.keySet()) { + dim1Vals.add(((int[]) row.getDim(0))[0]); } ArrayList dim2Vals = new ArrayList<>(); - for (IncrementalIndexRow row : toPersist1.getFacts().keySet()) { - dim2Vals.add(((int[]) row.getDims()[1])[0]); + for (IncrementalIndexRow row : (Iterable) toPersist1.keySet()) { + dim2Vals.add(((int[]) row.getDim(1))[0]); } final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java index eb1092c476ba..f406dfa20c62 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexRowCompTest.java @@ -22,10 +22,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.junit.Assert; import org.junit.Test; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Comparator; import java.util.Map; @@ -34,44 +36,131 @@ */ public class IncrementalIndexRowCompTest { + private static final Logger log = new Logger(IncrementalIndexRowCompTest.class); + @Test - public void testBasic() + public void testBasic() throws IndexSizeExceededException { - IncrementalIndex index = new IncrementalIndex.Builder() - .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) - .setMaxRowCount(1000) - .buildOnheap(); + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(new CountAggregatorFactory("cnt")) + .setMaxRowCount(1000) + .buildOnheap(); long time = System.currentTimeMillis(); - IncrementalIndexRow ir1 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); - IncrementalIndexRow ir2 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "A")).getIncrementalIndexRow(); - IncrementalIndexRow ir3 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A")).getIncrementalIndexRow(); + IncrementalIndexRow td1 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + IncrementalIndexRow td2 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "A")).getIncrementalIndexRow(); + IncrementalIndexRow td3 = index.toIncrementalIndexRow(toMapRow(time, "billy", "A")).getIncrementalIndexRow(); - IncrementalIndexRow ir4 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "B")).getIncrementalIndexRow(); - IncrementalIndexRow ir5 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", Arrays.asList("A", "B"))).getIncrementalIndexRow(); - IncrementalIndexRow ir6 = index.toIncrementalIndexRow(toMapRow(time + 1)).getIncrementalIndexRow(); + IncrementalIndexRow td4 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + IncrementalIndexRow td5 = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", Arrays.asList("A", "B"))).getIncrementalIndexRow(); + IncrementalIndexRow td6 = index.toIncrementalIndexRow(toMapRow(time + 1)).getIncrementalIndexRow(); Comparator comparator = index.dimsComparator(); - Assert.assertEquals(0, comparator.compare(ir1, ir1)); - Assert.assertEquals(0, comparator.compare(ir2, ir2)); - Assert.assertEquals(0, comparator.compare(ir3, ir3)); - - Assert.assertTrue(comparator.compare(ir1, ir2) > 0); - Assert.assertTrue(comparator.compare(ir2, ir1) < 0); - Assert.assertTrue(comparator.compare(ir2, ir3) > 0); - Assert.assertTrue(comparator.compare(ir3, ir2) < 0); - Assert.assertTrue(comparator.compare(ir1, ir3) > 0); - Assert.assertTrue(comparator.compare(ir3, ir1) < 0); - - Assert.assertTrue(comparator.compare(ir6, ir1) > 0); - Assert.assertTrue(comparator.compare(ir6, ir2) > 0); - Assert.assertTrue(comparator.compare(ir6, ir3) > 0); - - Assert.assertTrue(comparator.compare(ir4, ir6) > 0); - Assert.assertTrue(comparator.compare(ir5, ir6) > 0); - Assert.assertTrue(comparator.compare(ir4, ir5) < 0); - Assert.assertTrue(comparator.compare(ir5, ir4) > 0); + Assert.assertEquals(0, comparator.compare(td1, td1)); + Assert.assertEquals(0, comparator.compare(td2, td2)); + Assert.assertEquals(0, comparator.compare(td3, td3)); + + Assert.assertTrue(comparator.compare(td1, td2) > 0); + Assert.assertTrue(comparator.compare(td2, td1) < 0); + Assert.assertTrue(comparator.compare(td2, td3) > 0); + Assert.assertTrue(comparator.compare(td3, td2) < 0); + Assert.assertTrue(comparator.compare(td1, td3) > 0); + Assert.assertTrue(comparator.compare(td3, td1) < 0); + + Assert.assertTrue(comparator.compare(td6, td1) > 0); + Assert.assertTrue(comparator.compare(td6, td2) > 0); + Assert.assertTrue(comparator.compare(td6, td3) > 0); + + Assert.assertTrue(comparator.compare(td4, td6) > 0); + Assert.assertTrue(comparator.compare(td5, td6) > 0); + Assert.assertTrue(comparator.compare(td4, td5) < 0); + Assert.assertTrue(comparator.compare(td5, td4) > 0); + } + + @Test + public void testTimeAndDimsSerialization() throws IndexSizeExceededException + { + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(true, new CountAggregatorFactory("cnt")) + .setMaxRowCount(1000) + .buildOffheapOak(); + + OakIncrementalIndex oakIndex = (OakIncrementalIndex) index; + + long time = System.currentTimeMillis(); + IncrementalIndexRow[] origIncrementalIndexRow = new IncrementalIndexRow[6]; + ByteBuffer[] serializedIncrementalIndexRow = new ByteBuffer[6]; + IncrementalIndexRow[] deserializedIncrementalIndexRow = new IncrementalIndexRow[6]; + + Comparator comparator = index.dimsComparator(); + + origIncrementalIndexRow[0] = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + origIncrementalIndexRow[1] = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "A")).getIncrementalIndexRow(); + origIncrementalIndexRow[2] = index.toIncrementalIndexRow(toMapRow(time, "billy", "A")).getIncrementalIndexRow(); + origIncrementalIndexRow[3] = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + origIncrementalIndexRow[4] = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "A,B")).getIncrementalIndexRow(); + origIncrementalIndexRow[5] = index.toIncrementalIndexRow(toMapRow(time + 1)).getIncrementalIndexRow(); + + OakKeySerializer serializer = new OakKeySerializer(oakIndex.dimensionDescsList); + + for (int i = 0; i < 6; i++) { + serializedIncrementalIndexRow[i] = ByteBuffer.allocate(serializer.calculateSize(origIncrementalIndexRow[i])); + serializer.serialize(origIncrementalIndexRow[i], serializedIncrementalIndexRow[i]); + deserializedIncrementalIndexRow[i] = serializer.deserialize(serializedIncrementalIndexRow[i]); + Assert.assertEquals(0, comparator.compare(origIncrementalIndexRow[i], deserializedIncrementalIndexRow[i])); + } + } + + @Test + public void testIncrementalIndexRowByteBufferComparator() throws IndexSizeExceededException + { + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(true, new CountAggregatorFactory("cnt")) + .setMaxRowCount(1000) + .buildOffheapOak(); + + OakIncrementalIndex oakIndex = (OakIncrementalIndex) index; + + long time = System.currentTimeMillis(); + + IncrementalIndexRow[] incrementalIndexRowArray = new IncrementalIndexRow[6]; + incrementalIndexRowArray[0] = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + incrementalIndexRowArray[1] = index.toIncrementalIndexRow(toMapRow(time, "billy", "A", "joe", "A")).getIncrementalIndexRow(); + incrementalIndexRowArray[2] = index.toIncrementalIndexRow(toMapRow(time, "billy", "A")).getIncrementalIndexRow(); + incrementalIndexRowArray[3] = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "B")).getIncrementalIndexRow(); + incrementalIndexRowArray[4] = index.toIncrementalIndexRow(toMapRow(time + 1, "billy", "A", "joe", "B,A")).getIncrementalIndexRow(); + incrementalIndexRowArray[5] = index.toIncrementalIndexRow(toMapRow(time + 1)).getIncrementalIndexRow(); + + OakKeySerializer serializer = new OakKeySerializer(oakIndex.dimensionDescsList); + + ByteBuffer[] incrementalIndexRowByteBufferArray = new ByteBuffer[6]; + for (int i = 0; i < 6; i++) { + incrementalIndexRowByteBufferArray[i] = ByteBuffer.allocate(serializer.calculateSize(incrementalIndexRowArray[i])); + serializer.serialize(incrementalIndexRowArray[i], incrementalIndexRowByteBufferArray[i]); + } + + OakKeysComparator comparator = new OakKeysComparator(oakIndex.dimensionDescsList, true); + + Assert.assertEquals(0, comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[0], incrementalIndexRowByteBufferArray[0])); + Assert.assertEquals(0, comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[1], incrementalIndexRowByteBufferArray[1])); + Assert.assertEquals(0, comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[2], incrementalIndexRowByteBufferArray[2])); + + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[0], incrementalIndexRowByteBufferArray[1]) > 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[1], incrementalIndexRowByteBufferArray[0]) < 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[1], incrementalIndexRowByteBufferArray[2]) > 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[2], incrementalIndexRowByteBufferArray[1]) < 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[0], incrementalIndexRowByteBufferArray[2]) > 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[2], incrementalIndexRowByteBufferArray[0]) < 0); + + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[5], incrementalIndexRowByteBufferArray[0]) > 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[5], incrementalIndexRowByteBufferArray[1]) > 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[5], incrementalIndexRowByteBufferArray[2]) > 0); + + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[3], incrementalIndexRowByteBufferArray[5]) > 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[4], incrementalIndexRowByteBufferArray[5]) > 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[3], incrementalIndexRowByteBufferArray[4]) < 0); + Assert.assertTrue(comparator.compareSerializedKeys(incrementalIndexRowByteBufferArray[4], incrementalIndexRowByteBufferArray[3]) > 0); } private MapBasedInputRow toMapRow(long time, Object... dimAndVal) diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OakIncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/OakIncrementalIndexTest.java new file mode 100644 index 000000000000..cc0dc5a9782c --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OakIncrementalIndexTest.java @@ -0,0 +1,387 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.incremental; + +import com.google.common.collect.Maps; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.FilteredAggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; + +import java.util.function.Consumer; + +public class OakIncrementalIndexTest +{ + + private static final Logger log = new Logger(OakIncrementalIndexTest.class); + + @Test + public void testOffHeapOakIncrementalIndexBasics() throws Exception + { + OakIncrementalIndex index = getIndex(true); + MapBasedInputRow[] rows = new MapBasedInputRow[6]; + long minTime = System.currentTimeMillis() - 1000 * rows.length; + + // creating rows + rows[0] = toMapRow(minTime); + rows[1] = toMapRow(minTime + 1000, "StringA", "A"); + rows[2] = toMapRow(minTime + 1000, "StringA", "B"); + rows[3] = toMapRow(minTime + 4000, "StringA", "A", "StringB", "B", "StringC", "C", "StringD", "D"); + rows[4] = toMapRow(minTime + 2000, "StringA", "A", "StringB", "B"); + rows[5] = toMapRow(minTime + 3000, "StringD", "D"); + + for (MapBasedInputRow row : rows) { + for (int j = 0; j < 5; j++) { + index.add(row); + } + } + + Assert.assertEquals(index.size(), rows.length); + Assert.assertEquals(index.getMinTimeMillis(), minTime); + Assert.assertEquals(index.getMaxTimeMillis(), minTime + 4000); + index.close(); + } + + @Test + public void testOffHeapOakIncrementalIndexNoSchema() throws Exception + { + IncrementalIndex index = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(true, new CountAggregatorFactory("cnt")) + .setMaxRowCount(1000) + .buildOffheapOak(); + + MapBasedInputRow[] rows = new MapBasedInputRow[6]; + long minTime = System.currentTimeMillis() - 1000 * rows.length; + + // creating rows + rows[0] = toMapRow(minTime); + rows[1] = toMapRow(minTime + 1000, "StringA", "A"); + rows[2] = toMapRow(minTime + 1000, "StringA", "B"); + rows[3] = toMapRow(minTime + 4000, "StringA", "A", "StringB", "B", "StringC", "C", "StringD", "D"); + rows[4] = toMapRow(minTime + 2000, "StringA", "A", "StringB", "B"); + rows[5] = toMapRow(minTime + 3000, "StringD", "D"); + + for (MapBasedInputRow row : rows) { + for (int j = 0; j < 5; j++) { + index.add(row); + } + } + + Assert.assertEquals(index.size(), rows.length); + Assert.assertEquals(index.getMinTimeMillis(), minTime); + Assert.assertEquals(index.getMaxTimeMillis(), minTime + 4000); + Assert.assertEquals(index.getDimensionNames().size(), 4); + index.close(); + } + + @Test + public void testOffHeapOakIncrementalIndexKeysIterator() throws Exception + { + OakIncrementalIndex index = getIndex(true); + MapBasedInputRow[] rows = new MapBasedInputRow[10]; + + long time = System.currentTimeMillis(); + + // creating rows + rows[0] = toMapRow(time - 5000); + rows[1] = toMapRow(time - 3000); + rows[2] = toMapRow(time - 4000, "StringA", "A"); + rows[3] = toMapRow(time - 3000, "StringA", "ABC", "StringB", "B"); + rows[4] = toMapRow(time - 3000, "StringD", "A", "StringC", "B"); + rows[5] = toMapRow(time - 2000, "StringD", "D"); + rows[6] = toMapRow(time - 1000, "StringA", "AA", "StringB", "BH", "StringC", "C", "StringD", "D"); + rows[7] = toMapRow(time - 4000, "StringC", "C"); + rows[8] = toMapRow(time - 1000, "StringA", "C", "StringB", "A", "StringC", "BGD", "StringD", "B"); + rows[9] = toMapRow(time - 3000, "StringD", "A", "StringB", "B"); + + for (int j = 0; j < 5; j++) { + for (MapBasedInputRow row : rows) { + index.add(row); + } + } + Assert.assertEquals(index.size(), rows.length); + + Iterable keySet = index.keySet(); + Consumer keySetConsumer = new Consumer() { + + + IncrementalIndexRow prev = index.toIncrementalIndexRow(toMapRow(time - 6000)).getIncrementalIndexRow(); + + @Override + public void accept(IncrementalIndexRow timeAndDims) + { + Assert.assertTrue(0 > index.dimsComparator().compare(prev, timeAndDims)); + prev = timeAndDims; + } + }; + + keySet.forEach(keySetConsumer); + Assert.assertEquals(index.size(), rows.length); + index.close(); + } + + @Test + public void testOffHeapOakIncrementalIndexKeysTimeRangeIterable() throws Exception + { + OakIncrementalIndex index = getIndex(true); + MapBasedInputRow[] rows = new MapBasedInputRow[10]; + + long time = System.currentTimeMillis(); + + // creating rows + rows[0] = toMapRow(time - 5000); + rows[1] = toMapRow(time - 3000); + rows[2] = toMapRow(time - 4000, "StringA", "A"); + rows[3] = toMapRow(time - 3000, "StringA", "ABC", "StringB", "B"); + rows[4] = toMapRow(time - 3000, "StringD", "A", "StringC", "B"); + rows[5] = toMapRow(time - 2000, "StringD", "D"); + rows[6] = toMapRow(time - 1000, "StringA", "AA", "StringB", "BH", "StringC", "C", "StringD", "D"); + rows[7] = toMapRow(time - 3000, "StringB", "ABC", "StringA", "B"); + rows[8] = toMapRow(time - 1000, "StringA", "C", "StringB", "A", "StringC", "BGD", "StringD", "B"); + rows[9] = toMapRow(time - 3000, "StringD", "A", "StringB", "B"); + + for (int j = 0; j < 5; j++) { + for (MapBasedInputRow row : rows) { + index.add(row); + } + } + + Iterable timeRangeIterable; + Consumer timeRangeConsumer; + + // An ascending iterator + timeRangeIterable = index.timeRangeIterable(false, time - 5000, time - 2000); + timeRangeConsumer = new Consumer() { + + IncrementalIndexRow prev = index.toIncrementalIndexRow(toMapRow(time - 6000)).getIncrementalIndexRow(); + + @Override + public void accept(IncrementalIndexRow incrementalIndexRow) + { + Assert.assertTrue(0 > index.dimsComparator().compare(prev, incrementalIndexRow)); + Assert.assertTrue(time - 1999 > incrementalIndexRow.getTimestamp()); + Assert.assertTrue(time - 5001 < incrementalIndexRow.getTimestamp()); + prev = incrementalIndexRow; + } + }; + + timeRangeIterable.forEach(timeRangeConsumer); + + // A descending iterator + timeRangeIterable = index.timeRangeIterable(true, time - 5000, time - 2000); + timeRangeConsumer = new Consumer() { + + IncrementalIndexRow prev = index.toIncrementalIndexRow(toMapRow(time - 1000)).getIncrementalIndexRow(); + + @Override + public void accept(IncrementalIndexRow incrementalIndexRow) + { + Assert.assertTrue(0 < index.dimsComparator().compare(prev, incrementalIndexRow)); + Assert.assertTrue(time - 1999 > incrementalIndexRow.getTimestamp()); + Assert.assertTrue(time - 5001 < incrementalIndexRow.getTimestamp()); + prev = incrementalIndexRow; + } + }; + + timeRangeIterable.forEach(timeRangeConsumer); + Assert.assertEquals(index.size(), rows.length); + index.close(); + } + + @Test + public void testOffHeapOakIncrementalIndexAggs() throws Exception + { + OakIncrementalIndex index = getIndex(true); + MapBasedInputRow[] rows = new MapBasedInputRow[10]; + int insertionTrials = 5; + + long time = System.currentTimeMillis(); + + // creating rows + rows[0] = toMapRow(time - 5000); + rows[1] = toMapRow(time - 3000); + rows[2] = toMapRow(time - 4000, "StringA", "A"); + rows[3] = toMapRow(time - 3000, "StringA", "ABC", "StringB", "B"); + rows[4] = toMapRow(time - 3000, "StringD", "A", "StringC", "B"); + rows[5] = toMapRow(time - 2000, "StringD", "D"); + rows[6] = toMapRow(time - 1000, "StringA", "AA", "StringB", "BH", "StringC", "C", "StringD", "D"); + rows[7] = toMapRow(time - 3000, "StringB", "ABC", "StringA", "B"); + rows[8] = toMapRow(time - 1000, "StringA", "C", "StringB", "A", "StringC", "BGD", "StringD", "B"); + rows[9] = toMapRow(time - 3000, "StringD", "A", "StringB", "B"); + + for (int j = 0; j < insertionTrials; j++) { + for (MapBasedInputRow row : rows) { + index.add(row); + } + } + + Iterable iterable = index.iterableWithPostAggregations(null, false); + Consumer rowConsumer = new Consumer() { + + @Override + public void accept(Row row) + { + // insertion trials counters + long count = Long.valueOf(row.getDimension("Count").get(0)); + long countA = Long.valueOf(row.getDimension("CountStringA=A").get(0)); + long countB = Long.valueOf(row.getDimension("CountStringB=B").get(0)); + long countC = Long.valueOf(row.getDimension("CountStringC=C").get(0)); + long countD = Long.valueOf(row.getDimension("CountStringD=D").get(0)); + + Assert.assertEquals(insertionTrials, count); + Assert.assertEquals(countA, row.getDimension("StringA").size() > 0 && + row.getDimension("StringA").get(0).equals("A") ? insertionTrials : 0); + Assert.assertEquals(countB, row.getDimension("StringB").size() > 0 + && row.getDimension("StringB").get(0).equals("B") ? insertionTrials : 0); + Assert.assertEquals(countC, row.getDimension("StringC").size() > 0 && + row.getDimension("StringC").get(0).equals("C") ? insertionTrials : 0); + Assert.assertEquals(countD, row.getDimension("StringD").size() > 0 && + row.getDimension("StringD").get(0).equals("D") ? insertionTrials : 0); + } + }; + + iterable.forEach(rowConsumer); + Assert.assertEquals(index.size(), rows.length); + index.close(); + } + + @Test + public void testOffHeapOakIncrementalIndexPlainMode() throws Exception + { + OakIncrementalIndex index = getIndex(false); + MapBasedInputRow[] rows = new MapBasedInputRow[10]; + int insertionTrials = 5; + + long time = System.currentTimeMillis(); + + // creating rows + rows[0] = toMapRow(time); + rows[1] = toMapRow(time); + rows[2] = toMapRow(time, "StringA", "A"); + rows[3] = toMapRow(time, "StringA", "ABC", "StringB", "B"); + rows[4] = toMapRow(time, "StringD", "A", "StringC", "B"); + rows[5] = toMapRow(time, "StringD", "D"); + rows[6] = toMapRow(time, "StringA", "AA", "StringB", "BH", "StringC", "C", "StringD", "D"); + rows[7] = toMapRow(time, "StringB", "ABC", "StringA", "B"); + rows[8] = toMapRow(time, "StringA", "C", "StringB", "A", "StringC", "BGD", "StringD", "B"); + rows[9] = toMapRow(time, "StringD", "A", "StringB", "B"); + + for (int j = 0; j < insertionTrials; j++) { + for (MapBasedInputRow row : rows) { + index.add(row); + } + } + + Assert.assertEquals(index.size(), 50); + Iterable iterable = index.iterableWithPostAggregations(null, false); + Consumer rowConsumer = row -> { + // insertion trials counters + long count = Long.valueOf(row.getDimension("Count").get(0)); + long countA = Long.valueOf(row.getDimension("CountStringA=A").get(0)); + long countB = Long.valueOf(row.getDimension("CountStringB=B").get(0)); + long countC = Long.valueOf(row.getDimension("CountStringC=C").get(0)); + long countD = Long.valueOf(row.getDimension("CountStringD=D").get(0)); + + Assert.assertEquals(1, count); + Assert.assertEquals(countA, row.getDimension("StringA").size() > 0 && + row.getDimension("StringA").get(0).equals("A") ? 1 : 0); + Assert.assertEquals(countB, row.getDimension("StringB").size() > 0 && + row.getDimension("StringB").get(0).equals("B") ? 1 : 0); + Assert.assertEquals(countC, row.getDimension("StringC").size() > 0 && + row.getDimension("StringC").get(0).equals("C") ? 1 : 0); + Assert.assertEquals(countD, row.getDimension("StringD").size() > 0 && + row.getDimension("StringD").get(0).equals("D") ? 1 : 0); + }; + + iterable.forEach(rowConsumer); + Assert.assertEquals(index.size(), rows.length * insertionTrials); + index.close(); + } + + private OakIncrementalIndex getIndex(boolean rollup) + { + DimensionsSpec dimensions = new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("StringA"), + new StringDimensionSchema("StringB"), + new StringDimensionSchema("StringC"), + new StringDimensionSchema("StringD") + ), null, null + ); + + AggregatorFactory[] metrics = { + new CountAggregatorFactory("Count"), + new FilteredAggregatorFactory( + new CountAggregatorFactory("CountStringA=A"), + new SelectorDimFilter("StringA", "A", null) + ), + new FilteredAggregatorFactory( + new CountAggregatorFactory("CountStringB=B"), + new SelectorDimFilter("StringB", "B", null) + ), + new FilteredAggregatorFactory( + new CountAggregatorFactory("CountStringC=C"), + new SelectorDimFilter("StringC", "C", null) + ), + new FilteredAggregatorFactory( + new CountAggregatorFactory("CountStringD=D"), + new SelectorDimFilter("StringD", "D", null) + ) + }; + + final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() + .withDimensionsSpec(dimensions) + .withMetrics(metrics) + .withRollup(rollup) + .build(); + + OakIncrementalIndex index = (OakIncrementalIndex) new IncrementalIndex.Builder() + .setIndexSchema(schema) + .setDeserializeComplexMetrics(false) + .setMaxRowCount(1000) + .buildOffheapOak(); + + return index; + } + + private MapBasedInputRow toMapRow(long time, Object... dimAndVal) + { + Map data = Maps.newHashMap(); + List dims = new ArrayList<>(); + for (int i = 0; i < dimAndVal.length; i += 2) { + data.put((String) dimAndVal[i], dimAndVal[i + 1]); + dims.add((String) dimAndVal[i]); + } + + return new MapBasedInputRow(time, dims, data); + } +} 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 eb4609cb6881..7947240cbefe 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 @@ -156,31 +156,16 @@ public MapIncrementalIndex( ); } - @Override - protected Aggregator[] concurrentGet(int offset) - { - // All get operations should be fine - return indexedMap.get(offset); - } - - @Override - protected void concurrentSet(int offset, Aggregator[] value) - { - indexedMap.put(offset, value); - } - @Override protected AddToFactsResult addToFacts( - AggregatorFactory[] metrics, - boolean deserializeComplexMetrics, - boolean reportParseExceptions, - InputRow row, - AtomicInteger numEntries, - AtomicLong sizeInBytes, - IncrementalIndexRow key, - ThreadLocal rowContainer, - Supplier rowSupplier, - boolean skipMaxRowsInMemoryCheck // ignore for benchmark + boolean reportParseExceptions, + InputRow row, + AtomicInteger numEntries, + AtomicLong sizeInBytes, + IncrementalIndexRow key, + ThreadLocal rowContainer, + Supplier rowSupplier, + boolean skipMaxRowsInMemoryCheck // ignore for benchmark ) throws IndexSizeExceededException { @@ -189,26 +174,26 @@ protected AddToFactsResult addToFacts( Aggregator[] aggs; if (null != priorIdex) { - aggs = indexedMap.get(priorIdex); + aggs = aggsManager.concurrentGet(priorIdex); } else { - aggs = new Aggregator[metrics.length]; + aggs = new Aggregator[aggsManager.metrics.length]; - for (int i = 0; i < metrics.length; i++) { - final AggregatorFactory agg = metrics[i]; + for (int i = 0; i < aggsManager.metrics.length; i++) { + final AggregatorFactory agg = aggsManager.metrics[i]; aggs[i] = agg.factorize( - makeColumnSelectorFactory(agg, rowSupplier, deserializeComplexMetrics) + aggsManager.makeColumnSelectorFactory(agg, rowSupplier, aggsManager.deserializeComplexMetrics) ); } Integer rowIndex; do { rowIndex = indexIncrement.incrementAndGet(); - } while (null != indexedMap.putIfAbsent(rowIndex, aggs)); + } while (null != aggsManager.concurrentPutIfAbsent(rowIndex, aggs)); // Last ditch sanity checks if ((numEntries.get() >= maxRowCount || sizeInBytes.get() >= maxBytesInMemory) - && getFacts().getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX) { + && getFacts().getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX) { throw new IndexSizeExceededException("Maximum number of rows or max bytes reached"); } final int prev = getFacts().putIfAbsent(key, rowIndex); @@ -217,9 +202,9 @@ && getFacts().getPriorIndex(key) == IncrementalIndexRow.EMPTY_ROW_INDEX) { sizeInBytes.incrementAndGet(); } else { // We lost a race - aggs = indexedMap.get(prev); + aggs = aggsManager.concurrentGet(prev); // Free up the misfire - indexedMap.remove(rowIndex); + aggsManager.concurrentRemove(rowIndex); // This is expected to occur ~80% of the time in the worst scenarios } } diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java index 76ce9d76315b..a5afefe5895c 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/OnheapIncrementalIndexTest.java @@ -85,8 +85,9 @@ public void run() public void run() { while (!Thread.interrupted()) { - for (IncrementalIndexRow row : index.getFacts().keySet()) { - if (index.getMetricLongValue(row.getRowIndex(), 0) != 1) { + Iterable iterKeySet = index.keySet(); + for (IncrementalIndexRow row : iterKeySet) { + if (index.getMetricLongValue(row, 0) != 1) { checkFailedCount.addAndGet(1); } } @@ -128,7 +129,7 @@ public void testOnHeapIncrementalIndexClose() throws Exception )); // override the aggregators with the mocks - index.concurrentGet(0)[0] = mockedAggregator; + index.aggsManager.concurrentGet(0)[0] = mockedAggregator; // close the indexer and validate the expectations EasyMock.replay(mockedAggregator);