From e65066c311425683bb2e632fce4210540f01bd10 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Sun, 17 Feb 2019 14:05:03 +0200 Subject: [PATCH 01/15] Support Oak in druid --- .../IncrementalIndexReadBenchmark.java | 46 +- .../indexing/IndexIngestionBenchmark.java | 64 +- pom.xml | 2 +- processing/pom.xml | 5 + .../druid/segment/DoubleDimensionIndexer.java | 18 +- .../druid/segment/FloatDimensionIndexer.java | 15 +- .../druid/segment/LongDimensionIndexer.java | 15 +- .../druid/segment/StringDimensionIndexer.java | 21 +- .../segment/data/ArrayBasedIndexedInts.java | 8 + .../segment/incremental/IncrementalIndex.java | 57 +- .../incremental/IncrementalIndexAdapter.java | 5 +- .../incremental/IncrementalIndexRow.java | 26 +- .../incremental/OakIncrementalIndex.java | 693 ++++++++++++++++++ .../incremental/OakIncrementalIndexRow.java | 160 ++++ .../segment/incremental/OakKeySerializer.java | 151 ++++ .../incremental/OakKeysComparator.java | 221 ++++++ .../incremental/OakValueSerializer.java | 62 ++ .../incremental/OffheapIncrementalIndex.java | 20 +- .../incremental/OnheapIncrementalIndex.java | 20 +- .../IncrementalIndexAdapterTest.java | 4 +- .../OnheapIncrementalIndexTest.java | 2 +- 21 files changed, 1510 insertions(+), 105 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/OakValueSerializer.java 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..84445164f957 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,6 +57,7 @@ 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; @@ -82,6 +83,9 @@ public class IncrementalIndexReadBenchmark @Param({"true", "false"}) private boolean rollup; + @Param({"onheap", "oak"}) + private String indexType; + private static final Logger log = new Logger(IncrementalIndexReadBenchmark.class); private static final int RNG_SEED = 9999; private IncrementalIndex incIndex; @@ -118,18 +122,40 @@ public void setup() throws IOException } + @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) + .buildOak(); + + } + return null; } @Benchmark 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..702356ba13da 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) + .buildOak(); + + } + 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(1) + .build(); + + new Runner(opt).run(); + } + + } diff --git a/pom.xml b/pom.xml index ba9d71304587..2d9e5d4ce6bc 100644 --- a/pom.xml +++ b/pom.xml @@ -1220,7 +1220,7 @@ -Xmx1500m - -XX:MaxDirectMemorySize=512m + -XX:MaxDirectMemorySize=32g -Duser.language=en -Duser.GroupByQueryRunnerTest.javacountry=US -Dfile.encoding=UTF-8 diff --git a/processing/pom.xml b/processing/pom.xml index 1c469890ac49..66c007e110cc 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-core 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 0ddacfb781b0..d31f3dabb875 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java @@ -106,20 +106,19 @@ 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(); + final Object dim = currEntry.get().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 +126,13 @@ public double getDouble() @Override public Double getObject() { - final Object[] dims = currEntry.get().getDims(); + final Object dim = currEntry.get().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 a17ebea883d4..8d3696f006b7 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,13 @@ public float getFloat() @Override public Float getObject() { - final Object[] dims = currEntry.get().getDims(); + final Object dim = currEntry.get().getDim(dimIndex); - if (dimIndex >= dims.length) { + if (dim == null) { return null; } - return (Float) dims[dimIndex]; + return (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 6c69735046a3..be2afc66b444 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,13 @@ public long getLong() @Override public Long getObject() { - final Object[] dims = currEntry.get().getDims(); + final Object dim = currEntry.get().getDim(dimIndex); - if (dimIndex >= dims.length) { + if (dim == null) { return null; } - return (Long) dims[dimIndex]; + return (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 cf201e4f9b29..55bbbfb7b8fb 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -443,11 +443,10 @@ 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]; + if (dimIndex < currEntry.get().getDimsLength()) { + indices = (int[]) currEntry.get().getDim(dimIndex); } else { indices = null; } @@ -497,12 +496,11 @@ public ValueMatcher makeValueMatcher(final String value) @Override public boolean matches() { - Object[] dims = currEntry.get().getDims(); - if (dimIndex >= dims.length) { + if (dimIndex >= currEntry.get().getDimsLength()) { return value == null; } - int[] dimsInt = (int[]) dims[dimIndex]; + int[] dimsInt = (int[]) currEntry.get().getDim(dimIndex); if (dimsInt == null || dimsInt.length == 0) { return value == null; } @@ -543,12 +541,11 @@ public ValueMatcher makeValueMatcher(final Predicate predicate) @Override public boolean matches() { - Object[] dims = currEntry.get().getDims(); - if (dimIndex >= dims.length) { + if (dimIndex >= currEntry.get().getDimsLength()) { return matchNull; } - int[] dimsInt = (int[]) dims[dimIndex]; + int[] dimsInt = (int[]) currEntry.get().getDim(dimIndex); if (dimsInt == null || dimsInt.length == 0) { return matchNull; } @@ -628,12 +625,12 @@ public Object getObject() return null; } - Object[] dims = key.getDims(); - if (dimIndex >= dims.length) { + Object dim = key.getDim(dimIndex); + if (dim == null) { 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..1f4970e53ccb 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 @@ -73,6 +73,14 @@ 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/IncrementalIndex.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndex.java index 39df02406ded..8acf5612fd3c 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 @@ -430,7 +430,6 @@ public IncrementalIndex buildOnheap() if (maxRowCount <= 0) { throw new IllegalArgumentException("Invalid max row count: " + maxRowCount); } - return new OnheapIncrementalIndex( Objects.requireNonNull(incrementalIndexSchema, "incrementIndexSchema is null"), deserializeComplexMetrics, @@ -458,6 +457,21 @@ public IncrementalIndex buildOffheap(final NonBlockingPool bufferPoo Objects.requireNonNull(bufferPool, "bufferPool is null") ); } + + public IncrementalIndex buildOak() + { + 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() @@ -493,15 +507,15 @@ protected abstract AddToFactsResult addToFacts( protected abstract Object getAggVal(AggregatorType agg, int rowOffset, int aggPosition); - protected abstract float getMetricFloatValue(int rowOffset, int aggOffset); + protected abstract float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggOffset); - protected abstract long getMetricLongValue(int rowOffset, int aggOffset); + protected abstract long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggOffset); - protected abstract Object getMetricObjectValue(int rowOffset, int aggOffset); + protected abstract Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggOffset); - protected abstract double getMetricDoubleValue(int rowOffset, int aggOffset); + protected abstract double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggOffset); - protected abstract boolean isNull(int rowOffset, int aggOffset); + protected abstract boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggOffset); static class IncrementalIndexRowResult { @@ -1007,11 +1021,9 @@ public Iterator iterator() 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]; + for (int i = 0; i < incrementalIndexRow.getDimsLength(); ++i) { + Object dim = incrementalIndexRow.getDim(i); DimensionDesc dimensionDesc = dimensions.get(i); if (dimensionDesc == null) { continue; @@ -1491,7 +1503,7 @@ public LongMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metricI public long getLong() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricLongValue(currEntry.get().getRowIndex(), metricIndex); + return getMetricLongValue(currEntry.get(), metricIndex); } @Override @@ -1503,7 +1515,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + return IncrementalIndex.this.isNull(currEntry.get(), metricIndex); } } @@ -1528,7 +1540,7 @@ public ObjectMetricColumnSelector( @Override public Object getObject() { - return getMetricObjectValue(currEntry.get().getRowIndex(), metricIndex); + return getMetricObjectValue(currEntry.get(), metricIndex); } @Override @@ -1559,7 +1571,7 @@ public FloatMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metric public float getFloat() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricFloatValue(currEntry.get().getRowIndex(), metricIndex); + return getMetricFloatValue(currEntry.get(), metricIndex); } @Override @@ -1571,7 +1583,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + return IncrementalIndex.this.isNull(currEntry.get(), metricIndex); } } @@ -1590,13 +1602,13 @@ public DoubleMetricColumnSelector(IncrementalIndexRowHolder currEntry, int metri public double getDouble() { assert NullHandling.replaceWithDefault() || !isNull(); - return getMetricDoubleValue(currEntry.get().getRowIndex(), metricIndex); + return getMetricDoubleValue(currEntry.get(), metricIndex); } @Override public boolean isNull() { - return IncrementalIndex.this.isNull(currEntry.get().getRowIndex(), metricIndex); + return IncrementalIndex.this.isNull(currEntry.get(), metricIndex); } @Override @@ -1605,4 +1617,15 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) inspector.visit("index", IncrementalIndex.this); } } + + public List getDimensionDescsList() + { + //TODO YONIGO - is dimensionDescsList accessed concurrently? + return dimensionDescsList; + } + + public ThreadLocal getIn() + { + return in; + } } 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..cecdfffb2f23 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 @@ -94,14 +94,13 @@ private void processRows( { int rowNum = 0; for (IncrementalIndexRow row : index.getFacts().persistIterable()) { - final Object[] dims = row.getDims(); for (IncrementalIndex.DimensionDesc dimension : dimensions) { final int dimIndex = dimension.getIndex(); DimensionAccessor accessor = accessors.get(dimension.getName()); // Add 'null' to the dimension's dictionary. - if (dimIndex >= dims.length || dims[dimIndex] == null) { + if (dimIndex >= row.getDimsLength() || row.getDim(dimIndex) == null) { accessor.indexer.processRowValsToUnsortedEncodedKeyComponent(null, true); continue; } @@ -110,7 +109,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(row.getDim(dimIndex), 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..f7b497baf1d3 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,13 +30,13 @@ 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; + protected final List dimensionDescsList; /** * rowIndex is not checked in {@link #equals} and {@link #hashCode} on purpose. IncrementalIndexRow acts as a Map key @@ -82,6 +82,15 @@ private IncrementalIndexRow( this.dimsKeySize = dimsKeySize; } + public IncrementalIndexRow(int timestamp, List dimensionDescsList) + { + // Used by OakIncrementalIndex, probablly better to create an interface for both classes + // but I tried to avoid many code changes. + this.timestamp = timestamp; + dims = null; + this.dimensionDescsList = dimensionDescsList; + } + static IncrementalIndexRow createTimeAndDimswithDimsKeySize( long timestamp, Object[] dims, @@ -97,11 +106,20 @@ public long getTimestamp() return timestamp; } - public Object[] getDims() + public Object getDim(int index) { - return dims; + if (dims == null || index >= dims.length) { + return null; + } + return dims[index]; } + public int getDimsLength() + { + return dims == null ? 0 : dims.length; + } + + public int getRowIndex() { return rowIndex; 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..6e93342aba42 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java @@ -0,0 +1,693 @@ +/* + * 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 com.oath.oak.OakBufferView; +import com.oath.oak.OakIterator; +import com.oath.oak.OakMap; +import com.oath.oak.OakMapBuilder; +import com.oath.oak.OakTransformView; +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.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.query.aggregation.PostAggregator; +import org.apache.druid.segment.ColumnSelectorFactory; +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.reflect.generics.reflectiveObjects.NotImplementedException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; +import java.util.function.Function; + + + +/** + */ +public class OakIncrementalIndex extends IncrementalIndex +{ + + 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; + + + private final OakFactsHolder facts; + private AggsManager aggsManager; + + private final boolean reportParseExceptions; + private static final Logger log = new Logger(OakIncrementalIndex.class); + private String outOfRowsReason; + private final List dimensionDescsList; + + public OakIncrementalIndex(IncrementalIndexSchema incrementalIndexSchema, + boolean deserializeComplexMetrics, + boolean reportParseExceptions, + boolean concurrentEventAdd, + int maxRowCount) + { + super(incrementalIndexSchema, deserializeComplexMetrics, reportParseExceptions, concurrentEventAdd); + facts = new OakFactsHolder(incrementalIndexSchema, getDimensionDescsList(), aggsManager, getIn(), + incrementalIndexSchema.isRollup(), maxRowCount); + this.dimensionDescsList = getDimensionDescsList(); + this.reportParseExceptions = reportParseExceptions; + } + + @Override + public FactsHolder getFacts() + { + return facts; + } + + @Override + public boolean canAppendRow() + { + final boolean canAdd = size() < facts.getMaxRowCount(); + if (!canAdd) { + outOfRowsReason = StringUtils.format("Maximum number of rows [%d] reached", facts.getMaxRowCount()); + } + return canAdd; + } + + @Override + public String getOutOfRowsReason() + { + return outOfRowsReason; + } + + @Override + protected BufferAggregator[] initAggs(AggregatorFactory[] metrics, + Supplier rowSupplier, + boolean deserializeComplexMetrics, + boolean concurrentEventAdd) + { + Map selectors = new HashMap<>(); + int[] 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(); + } + } + + int aggsTotalSize; + if (metrics.length == 0) { + aggsTotalSize = 0; + } else { + aggsTotalSize = aggOffsetInBuffer[metrics.length - 1] + metrics[metrics.length - 1].getMaxIntermediateSizeWithNulls(); + } + + BufferAggregator[] aggs = new BufferAggregator[metrics.length]; + aggsManager = new AggsManager(aggs, selectors, aggOffsetInBuffer, aggsTotalSize, metrics, reportParseExceptions); + return aggs; + } + + @Override + public void close() + { + facts.close(); + } + @Override + protected AddToFactsResult addToFacts(AggregatorFactory[] metrics, + boolean deserializeComplexMetrics, + boolean reportParseExceptions, + InputRow row, + AtomicInteger numEntries, + AtomicLong sizeInBytes, + IncrementalIndexRow key, + ThreadLocal rowContainer, + Supplier rowSupplier, + boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException + { + return facts.addToOak(row, numEntries, key, rowContainer, skipMaxRowsInMemoryCheck); + } + + @Override + public int getLastRowIndex() + { + return this.facts.getRowIndexGenerator() - 1; + } + + @Override + protected BufferAggregator[] getAggsForRow(int rowOffset) + { + // We should never get here because we override iterableWithPostAggregations + throw new NotImplementedException(); + } + + @Override + protected Object getAggVal(BufferAggregator agg, int rowOffset, int aggPosition) + { + // We should never get here because we override iterableWithPostAggregations + // oakII doesnt have different structures to keep rowOffset + throw new NotImplementedException(); + } + + @Override + protected float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) + { + return facts.getMetricFloatValue(incrementalIndexRow, aggOffset); + } + + @Override + protected long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) + { + return facts.getMetricLongValue(incrementalIndexRow, aggOffset); + } + + @Override + protected Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) + { + return facts.getMetricObjectValue(incrementalIndexRow, aggOffset); + } + + @Override + protected double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) + { + return facts.getMetricDoubleValue(incrementalIndexRow, aggOffset); + } + + @Override + protected boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggOffset) + { + return facts.isNull(incrementalIndexRow, aggOffset); + } + + @Override + public Iterable iterableWithPostAggregations(final List postAggs, final boolean descending) + { + Function, Row> transformer = 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); + }; + + return () -> { + OakIterator iterator = facts.transformIterator(descending, transformer); + return Iterators.transform(iterator, row -> row); + }; + } + +//static methods + + 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 long getTimestamp(ByteBuffer buff) + { + return buff.getLong(buff.position() + TIME_STAMP_INDEX); + } + + static int getRowIndex(ByteBuffer buff) + { + return buff.getInt(buff.position() + ROW_INDEX_INDEX); + } + + 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 Object getDimValue(ByteBuffer buff, int dimIndex) + { + int dimsLength = getDimsLength(buff); + return getDimValue(buff, dimIndex, dimsLength); + } + + 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) + { + 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 class AggsManager + { + private final AggregatorFactory[] metrics; + private final ReentrantLock[] aggLocks; + private volatile Map selectors; + private final boolean reportParseExceptions; + + //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 final BufferAggregator[] aggs; + + public AggsManager(BufferAggregator[] aggs, + Map selectors, + int[] aggOffsetInBuffer, + int aggsTotalSize, AggregatorFactory[] metrics, + boolean reportParseExceptions) + { + this.aggs = aggs; + this.selectors = selectors; + this.aggOffsetInBuffer = aggOffsetInBuffer; + this.aggsTotalSize = aggsTotalSize; + this.metrics = metrics; + this.aggLocks = new ReentrantLock[this.aggs.length]; + for (int i = 0; i < this.aggLocks.length; i++) { + this.aggLocks[i] = new ReentrantLock(true); + } + this.reportParseExceptions = reportParseExceptions; + } + + public void initValue(ByteBuffer byteBuffer, + InputRow row, + ThreadLocal rowContainer) + { + if (metrics.length > 0 && aggs[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]; + 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(row, rowContainer, byteBuffer); + } + + public void aggregate( + InputRow row, + ThreadLocal rowContainer, + ByteBuffer aggBuffer + ) + { + rowContainer.set(row); + + for (int i = 0; i < metrics.length; i++) { + final BufferAggregator agg = aggs[i]; + + 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) { + //TODO YONIGO - is this the right bevaviour? not the same in on/offheap + 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); + } + + public int aggsTotalSize() + { + return aggsTotalSize; + } + + public BufferAggregator[] getAggs() + { + return aggs; + } + } + + + private static class OakFactsHolder implements FactsHolder + { + private final OakMap oak; + + private final long minTimestamp; + private final List dimensionDescsList; + private final boolean rollup; + private final int maxRowCount; + + private final AtomicInteger rowIndexGenerator; + + private final AggsManager aggsManager; + + public OakFactsHolder(IncrementalIndexSchema incrementalIndexSchema, + List dimensionDescsList, + AggsManager aggsManager, ThreadLocal in, + boolean rollup, + int maxRowCount) + { + OakMapBuilder builder = new OakMapBuilder<>(); + builder.setComparator(new OakKeysComparator(dimensionDescsList, rollup)) + .setKeySerializer(new OakKeySerializer(dimensionDescsList)) + .setValueSerializer(new OakValueSerializer(aggsManager, in)) + .setMinKey(getMinIncrementalIndexRow()); + oak = builder.build(); + this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); + this.dimensionDescsList = dimensionDescsList; + this.rollup = rollup; + this.maxRowCount = maxRowCount; + this.rowIndexGenerator = new AtomicInteger(0); + this.aggsManager = aggsManager; + } + + + public OakIterator transformIterator(boolean descending, Function, Row> transformer) + { + try (OakMap tmpOakMap = descending ? oak.descendingMap() : oak; + OakTransformView transformView = tmpOakMap.createTransformView(transformer)) { + OakIterator valuesIterator = transformView.entriesIterator(); + return valuesIterator; + } + } + + + private IncrementalIndexRow getMinIncrementalIndexRow() + { + return new IncrementalIndexRow(minTimestamp, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); + } + + @Override + public int getPriorIndex(IncrementalIndexRow key) + { + return 0; + } + + @Override + public long getMinTimeMillis() + { + return oak.getMinKey().getTimestamp(); + } + + @Override + public long getMaxTimeMillis() + { + return oak.getMaxKey().getTimestamp(); + } + + @Override + public Iterator iterator(boolean descending) + { + // We should never get here because we override iterableWithPostAggregations + throw new NotImplementedException(); + } + + @Override + public Iterable timeRangeIterable(boolean descending, long timeStart, long timeEnd) + { + if (timeStart > timeEnd) { + return null; + } + return () -> { + IncrementalIndexRow from = new IncrementalIndexRow(timeStart, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); + IncrementalIndexRow to = new IncrementalIndexRow(timeEnd, 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() + { + return () -> { + //TODO YONIGO - why not return oakIncrementalIndex and not have this diserialization? + OakIterator keysIterator = oak.keysIterator(); + return Iterators.transform(keysIterator, key -> key); + }; + } + + @Override + public Iterable persistIterable() + { + return keySet(); + } + + @Override + public int putIfAbsent(IncrementalIndexRow key, int rowIndex) + { + //Oak is pigibacking the FactsHolder and doesnt really have rowIndex stored in it. + throw new NotImplementedException(); + } + + @Override + public void clear() + { + //TODO YONIGO - add clear to oak + //oak.clear(); + } + + private AddToFactsResult addToOak( + InputRow row, + AtomicInteger numEntries, + IncrementalIndexRow incrementalIndexRow, + ThreadLocal rowContainer, + boolean skipMaxRowsInMemoryCheck + ) throws IndexSizeExceededException + { + + Consumer computer = buffer -> aggsManager.aggregate(row, rowContainer, buffer); + incrementalIndexRow.setRowIndex(rowIndexGenerator.getAndIncrement()); + boolean added = oak.putIfAbsentComputeIfPresent(incrementalIndexRow, row, computer); + if (added) { + numEntries.incrementAndGet(); + } + + //TODO YONIGO - we will continue to add and throw exceptions. + if ((numEntries.get() > maxRowCount) //TODO YONIGO: || sizeInBytes.get() >= maxBytesInMemory + && !skipMaxRowsInMemoryCheck) { + throw new IndexSizeExceededException( + "Maximum number of rows [%d] or max size in bytes [%d] reached", + maxRowCount + ); + } + + return new AddToFactsResult(oak.entries(), 0, new ArrayList<>()); + } + + public int getMaxRowCount() + { + return maxRowCount; + } + + public int getRowIndexGenerator() + { + return rowIndexGenerator.get(); + } + + public void close() + { + //TODO YONIGO - aggregators maybe should be closed like on/offheapindex? + oak.close(); + } + + protected float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + //TODO YONIGO - all these get functions are bad!. we should somehow put the aggregator bytebuffer + // in the incrementalindexrow to avoid and additional search in oak. + Function, Float> transformer = 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); + } + } + + + protected long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Long> transformer = 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); + } + } + + + protected Object getMetricObjectValue(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); + } + } + + + protected double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Double> transformer = 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); + } + } + + + protected boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggIndex) + { + Function, Boolean> transformer = 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); + } + } + } +} 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..66a344dbba5e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -0,0 +1,160 @@ +/* + * 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; + + +//wrapper class for bytebuffers +public class OakIncrementalIndexRow extends IncrementalIndexRow +{ + private final ByteBuffer byteBuffer; + private Integer dimsLength; + + public OakIncrementalIndexRow(ByteBuffer byteBuffer, List dimensionDescsList) + { + super(0, dimensionDescsList); + this.byteBuffer = byteBuffer; + this.dimsLength = null; // lazy initialization + } + + @Override + public long getTimestamp() + { + return byteBuffer.getLong(byteBuffer.position() + OakIncrementalIndex.TIME_STAMP_INDEX); + } + + @Override + public int getDimsLength() + { + // Read length only once + if (dimsLength == null) { + dimsLength = byteBuffer.getInt(byteBuffer.position() + OakIncrementalIndex.DIMS_LENGTH_INDEX); + } + return dimsLength; + } + + @Override + public Object getDim(int dimIndex) + { + if (dimIndex >= getDimsLength()) { + return null; + } + return getDimValue(dimIndex); + } + + @Override + public int getRowIndex() + { + return byteBuffer.getInt(byteBuffer.position() + OakIncrementalIndex.ROW_INDEX_INDEX); + } + + @Override + void setRowIndex(int rowIndex) + { + throw new UnsupportedOperationException(); + } + + /** + * 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 = byteBuffer.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 byteBuffer.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 = byteBuffer.getInt(dimIndexInBuffer); + if (dimType == OakIncrementalIndex.NO_DIM) { + return null; + } else if (dimType == ValueType.DOUBLE.ordinal()) { + dimObject = byteBuffer.getDouble(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.FLOAT.ordinal()) { + dimObject = byteBuffer.getFloat(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.LONG.ordinal()) { + dimObject = byteBuffer.getLong(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + } else if (dimType == ValueType.STRING.ordinal()) { + int arrayIndexOffset = byteBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_INDEX_OFFSET); + int arrayIndex = byteBuffer.position() + arrayIndexOffset; + int arraySize = byteBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + int[] array = new int[arraySize]; + for (int i = 0; i < arraySize; i++) { + array[i] = byteBuffer.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 byteBuffer.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..f33c81d950b5 --- /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 com.oath.oak.OakSerializer; +import org.apache.druid.segment.column.ValueType; +import java.nio.ByteBuffer; +import java.util.List; + + +public class OakKeySerializer implements OakSerializer +{ + + private final List dimensionDescsList; + + public OakKeySerializer(List dimensionDescsList) + { + this.dimensionDescsList = dimensionDescsList; + } + + @Override + public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBuffer) + { + long timestamp = incrementalIndexRow.getTimestamp(); + + int dimsLength = incrementalIndexRow.getDimsLength(); + int rowIndex = incrementalIndexRow.getRowIndex(); + + // calculating buffer indexes for writing the key data + int buffIndex = byteBuffer.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 + + byteBuffer.putLong(timeStampIndex, timestamp); + byteBuffer.putInt(dimsLengthIndex, dimsLength); + byteBuffer.putInt(rowIndexIndex, rowIndex); + for (int i = 0; i < dimsLength; i++) { + ValueType valueType = OakIncrementalIndex.getDimValueType(i, dimensionDescsList); + if (valueType == null || incrementalIndexRow.getDim(i) == null) { + byteBuffer.putInt(dimsIndex, noDim); + } else { + byteBuffer.putInt(dimsIndex + valueTypeOffset, valueType.ordinal()); + switch (valueType) { + case FLOAT: + byteBuffer.putFloat(dimsIndex + dataOffset, (Float) incrementalIndexRow.getDim(i)); + break; + case DOUBLE: + byteBuffer.putDouble(dimsIndex + dataOffset, (Double) incrementalIndexRow.getDim(i)); + break; + case LONG: + byteBuffer.putLong(dimsIndex + dataOffset, (Long) incrementalIndexRow.getDim(i)); + break; + case STRING: + int[] arr = (int[]) incrementalIndexRow.getDim(i); + byteBuffer.putInt(dimsIndex + arrayIndexOffset, dimsArrayOffset); + byteBuffer.putInt(dimsIndex + arrayLengthOffset, arr.length); + for (int arrIndex = 0; arrIndex < arr.length; arrIndex++) { + byteBuffer.putInt(dimsArraysIndex + arrIndex * Integer.BYTES, arr[arrIndex]); + } + dimsArraysIndex += (arr.length * Integer.BYTES); + dimsArrayOffset += (arr.length * Integer.BYTES); + break; + default: + byteBuffer.putInt(dimsIndex, noDim); + } + } + + dimsIndex += dimCapacity; + } + + } + + @Override + public IncrementalIndexRow deserialize(ByteBuffer byteBuffer) + { + long timeStamp = OakIncrementalIndex.getTimestamp(byteBuffer); + int dimsLength = OakIncrementalIndex.getDimsLength(byteBuffer); + int rowIndex = OakIncrementalIndex.getRowIndex(byteBuffer); + Object[] dims = new Object[dimsLength]; + for (int dimIndex = 0; dimIndex < dimsLength; dimIndex++) { + Object dim = OakIncrementalIndex.getDimValue(byteBuffer, dimIndex, dimsLength); + dims[dimIndex] = dim; + } + return new IncrementalIndexRow(timeStamp, dims, dimensionDescsList, rowIndex); + } + + @Override + public int calculateSize(IncrementalIndexRow incrementalIndexRow) + { + //TODO - YONIGO befrore this what == null is it correct now? + if (incrementalIndexRow.getDimsLength() == 0) { + 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 < incrementalIndexRow.getDimsLength(); i++) { + Object dim = incrementalIndexRow.getDim(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 * incrementalIndexRow.getDimsLength() + 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..47f73fa7c978 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java @@ -0,0 +1,221 @@ +/* + * 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 com.oath.oak.OakComparator; +import org.apache.druid.segment.DimensionIndexer; + + +import java.nio.ByteBuffer; +import java.util.List; + +public class OakKeysComparator implements OakComparator +{ + + private final List dimensionDescsList; + private final 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) { + return lastCompare(lhs.getRowIndex(), rhs.getRowIndex()); + } + if (lengthDiff > 0) { + // lhs has bigger dims + if (allNull(lhs, numComparisons)) { + return lastCompare(lhs.getRowIndex(), rhs.getRowIndex()); + } + } else { + // rhs has bigger dims + if (allNull(rhs, numComparisons)) { + return lastCompare(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) { + return lastCompare(OakIncrementalIndex.getRowIndex(lhs), OakIncrementalIndex.getRowIndex(rhs)); + } + if (lengthDiff > 0) { + // lhs has bigger dims + if (OakIncrementalIndex.checkDimsAllNull(lhs, numComparisons)) { + return lastCompare(OakIncrementalIndex.getRowIndex(lhs), OakIncrementalIndex.getRowIndex(rhs)); + } + } else { + // rhs has bigger dims + if (OakIncrementalIndex.checkDimsAllNull(rhs, numComparisons)) { + return lastCompare(OakIncrementalIndex.getRowIndex(lhs), OakIncrementalIndex.getRowIndex(rhs)); + } + } + return lengthDiff; + } + + return retVal; + } + + private int lastCompare(int lsIndex, int rsIndex) + { + if (!rollup || lsIndex == IncrementalIndexRow.EMPTY_ROW_INDEX || rsIndex == IncrementalIndexRow.EMPTY_ROW_INDEX) { + // If we are not rollup then keys shouldnt collide. + // If on of the keys is EMPTY_ROW_INDEX this is a lower or upper bound key and must be compared. + return lsIndex - rsIndex; + } else { + return 0; + } + } + + + @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) { + return lastCompare(OakIncrementalIndex.getRowIndex(lhs), rhs.getRowIndex()); + } + if (lengthDiff > 0) { + // lhs has bigger dims + if (OakIncrementalIndex.checkDimsAllNull(lhs, numComparisons)) { + return lastCompare(OakIncrementalIndex.getRowIndex(lhs), rhs.getRowIndex()); + } + } else { + // rhs has bigger dims + if (allNull(rhs, numComparisons)) { + return lastCompare(OakIncrementalIndex.getRowIndex(lhs), rhs.getRowIndex()); + } + } + return lengthDiff; + } + return retVal; + } + + private static boolean allNull(IncrementalIndexRow row, int startPosition) + { + for (int i = startPosition; i < row.getDimsLength(); i++) { + if (row.getDim(i) != null) { + return false; + } + } + return true; + } + + +} 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..8f331149a3ad --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakValueSerializer.java @@ -0,0 +1,62 @@ +/* + * 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.oath.oak.OakSerializer; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.Row; + +import java.nio.ByteBuffer; + + +public class OakValueSerializer implements OakSerializer +{ + + private final OakIncrementalIndex.AggsManager aggsManager; + private final ThreadLocal rowContainer; + + public OakValueSerializer(OakIncrementalIndex.AggsManager aggsManager, + ThreadLocal rowContainer) + { + this.aggsManager = aggsManager; + this.rowContainer = rowContainer; + } + + @Override + public void serialize(Row row, ByteBuffer byteBuffer) + { + //TODO YONIGO - why cast? + aggsManager.initValue(byteBuffer, (InputRow) row, rowContainer); + } + + @Override + public Row deserialize(ByteBuffer byteBuffer) + { + // cannot be deserialized without the IncrementalIndexRow + throw new UnsupportedOperationException(); + + } + + @Override + public int calculateSize(Row row) + { + return aggsManager.aggsTotalSize(); + } +} 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 95c88fc9606a..f729d621d3e6 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 @@ -277,46 +277,46 @@ protected Object getAggVal(BufferAggregator agg, int rowOffset, int aggPosition) } @Override - public float getMetricFloatValue(int rowOffset, int aggOffset) + public float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); return agg.getFloat(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); } @Override - public long getMetricLongValue(int rowOffset, int aggOffset) + public long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); return agg.getLong(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); } @Override - public Object getMetricObjectValue(int rowOffset, int aggOffset) + public Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); return agg.get(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); } @Override - public double getMetricDoubleValue(int rowOffset, int aggOffset) + public double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); return agg.getDouble(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); } @Override - public boolean isNull(int rowOffset, int aggOffset) + public boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggOffset) { BufferAggregator agg = getAggs()[aggOffset]; - int[] indexAndOffset = indexAndOffsets.get(rowOffset); + int[] indexAndOffset = indexAndOffsets.get(incrementalIndexRow.getRowIndex()); ByteBuffer bb = aggBuffers.get(indexAndOffset[0]).get(); return agg.isNull(bb, indexAndOffset[1] + aggOffsetInBuffer[aggOffset]); } 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 80e21a08c493..eea9f26c0845 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 @@ -337,33 +337,33 @@ protected Object getAggVal(Aggregator agg, int rowOffset, int aggPosition) } @Override - public float getMetricFloatValue(int rowOffset, int aggOffset) + public float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { - return concurrentGet(rowOffset)[aggOffset].getFloat(); + return concurrentGet(incrementalIndexRow.getRowIndex())[aggOffset].getFloat(); } @Override - public long getMetricLongValue(int rowOffset, int aggOffset) + public long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { - return concurrentGet(rowOffset)[aggOffset].getLong(); + return concurrentGet(incrementalIndexRow.getRowIndex())[aggOffset].getLong(); } @Override - public Object getMetricObjectValue(int rowOffset, int aggOffset) + public Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { - return concurrentGet(rowOffset)[aggOffset].get(); + return concurrentGet(incrementalIndexRow.getRowIndex())[aggOffset].get(); } @Override - protected double getMetricDoubleValue(int rowOffset, int aggOffset) + protected double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggOffset) { - return concurrentGet(rowOffset)[aggOffset].getDouble(); + return concurrentGet(incrementalIndexRow.getRowIndex())[aggOffset].getDouble(); } @Override - public boolean isNull(int rowOffset, int aggOffset) + public boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggOffset) { - return concurrentGet(rowOffset)[aggOffset].isNull(); + return concurrentGet(incrementalIndexRow.getRowIndex())[aggOffset].isNull(); } /** 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..4dd4a217f228 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 @@ -101,11 +101,11 @@ public void testGetRowsIterableNoRollup() throws Exception ArrayList dim1Vals = new ArrayList<>(); for (IncrementalIndexRow row : toPersist1.getFacts().keySet()) { - dim1Vals.add(((int[]) row.getDims()[0])[0]); + dim1Vals.add(((int[]) row.getDim(0))[0]); } ArrayList dim2Vals = new ArrayList<>(); for (IncrementalIndexRow row : toPersist1.getFacts().keySet()) { - dim2Vals.add(((int[]) row.getDims()[1])[0]); + dim2Vals.add(((int[]) row.getDim(1))[0]); } final IndexableAdapter incrementalAdapter = new IncrementalIndexAdapter( 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..5fa37209eefb 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 @@ -86,7 +86,7 @@ public void run() { while (!Thread.interrupted()) { for (IncrementalIndexRow row : index.getFacts().keySet()) { - if (index.getMetricLongValue(row.getRowIndex(), 0) != 1) { + if (index.getMetricLongValue(row, 0) != 1) { checkFailedCount.addAndGet(1); } } From f4eefd31e138db91105e4779a07860b8641b322b Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Sun, 17 Mar 2019 14:28:51 +0200 Subject: [PATCH 02/15] Save OakRbuffer in OakIncrementalIndexRow --- .../IncrementalIndexReadBenchmark.java | 36 ++++------- .../indexing/IndexIngestionBenchmark.java | 45 ++++---------- .../benchmark/query/SearchBenchmark.java | 20 +++++-- .../druid/benchmark/query/TopNBenchmark.java | 16 ++++- .../incremental/OakIncrementalIndex.java | 59 +++++++++---------- .../incremental/OakIncrementalIndexRow.java | 44 ++++++++------ 6 files changed, 106 insertions(+), 114 deletions(-) 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 84445164f957..c501dd494610 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 @@ -83,7 +83,7 @@ public class IncrementalIndexReadBenchmark @Param({"true", "false"}) private boolean rollup; - @Param({"onheap", "oak"}) + @Param({"oak", "onheap"}) private String indexType; private static final Logger log = new Logger(IncrementalIndexReadBenchmark.class); @@ -130,30 +130,20 @@ public void tearDown() private IncrementalIndex makeIncIndex() { + IncrementalIndex.Builder builder = new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment); switch (indexType) { case "onheap": - return new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMetrics(schemaInfo.getAggsArray()) - .withRollup(rollup) - .build() - ) - .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + builder.buildOnheap(); case "oak": - return new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMetrics(schemaInfo.getAggsArray()) - .withRollup(rollup) - .build() - ) - .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment) - .buildOak(); - + builder.buildOak(); } return null; } @@ -221,7 +211,7 @@ public void readWithFilters(Blackhole blackhole) private Sequence makeCursors(IncrementalIndexStorageAdapter sa, DimFilter filter) { return sa.makeCursors( - filter.toFilter(), + filter == null ? null : filter.toFilter(), schemaInfo.getDataInterval(), VirtualColumns.EMPTY, Granularities.ALL, 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 702356ba13da..badc709d1505 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 @@ -114,30 +114,20 @@ public void tearDown() private IncrementalIndex makeIncIndex() { + IncrementalIndex.Builder builder = new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment * 2); switch (indexType) { case "onheap": - return new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMetrics(schemaInfo.getAggsArray()) - .withRollup(rollup) - .build() - ) - .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment * 2) - .buildOnheap(); + return builder.buildOnheap(); case "oak": - return new IncrementalIndex.Builder() - .setIndexSchema( - new IncrementalIndexSchema.Builder() - .withMetrics(schemaInfo.getAggsArray()) - .withRollup(rollup) - .build() - ) - .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment * 2) - .buildOak(); - + return builder.buildOak(); } return null; } @@ -154,17 +144,4 @@ public void addRows(Blackhole blackhole) throws Exception } } - - public static void main(String[] args) throws RunnerException - { - Options opt = new OptionsBuilder() - .include(IndexIngestionBenchmark.class.getSimpleName()) - .threads(1) - .forks(1) - .build(); - - new Runner(opt).run(); - } - - } diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java index 2c5617e937b8..658061a719fa 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SearchBenchmark.java @@ -121,6 +121,9 @@ public class SearchBenchmark @Param({"1000"}) private int limit; + @Param({"oak", "onheap"}) + private String indexType; + private static final Logger log = new Logger(SearchBenchmark.class); private static final IndexMergerV9 INDEX_MERGER_V9; private static final IndexIO INDEX_IO; @@ -383,15 +386,22 @@ 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(); + IncrementalIndex.Builder builder = new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment); + switch (indexType) { + case "onheap": + return builder.buildOnheap(); + case "oak": + return builder.buildOak(); + } + return null; } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) 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 fee4d429264d..693eeb12b5a0 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 @@ -112,6 +112,9 @@ public class TopNBenchmark @Param({"10"}) private int threshold; + @Param({"oak", "onheap"}) + 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; @@ -290,15 +293,22 @@ 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() + IncrementalIndex.Builder builder = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) .setReportParseExceptions(false) - .setMaxRowCount(rowsPerSegment) - .buildOnheap(); + .setMaxRowCount(rowsPerSegment); + switch (indexType) { + case "onheap": + return builder.buildOnheap(); + case "oak": + return builder.buildOak(); + } + return null; } private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) 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 index 6e93342aba42..873b1d7d6505 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java @@ -27,6 +27,7 @@ import com.oath.oak.OakIterator; import com.oath.oak.OakMap; import com.oath.oak.OakMapBuilder; +import com.oath.oak.OakRBuffer; import com.oath.oak.OakTransformView; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedRow; @@ -234,6 +235,7 @@ protected boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggOffset) @Override public Iterable iterableWithPostAggregations(final List postAggs, final boolean descending) { + //TODO YONIGO - rewrite this function. maybe return an unserialized row? Function, Row> transformer = entry -> { ByteBuffer serializedKey = entry.getKey(); ByteBuffer serializedValue = entry.getValue(); @@ -540,8 +542,9 @@ public Iterable timeRangeIterable(boolean descending, long 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)); + OakIterator> iterator = bufferView.entriesIterator(); + return Iterators.transform(iterator, entry -> + new OakIncrementalIndexRow(entry.getKey(), dimensionDescsList, entry.getValue())); } }; } @@ -550,9 +553,12 @@ public Iterable timeRangeIterable(boolean descending, long public Iterable keySet() { return () -> { - //TODO YONIGO - why not return oakIncrementalIndex and not have this diserialization? - OakIterator keysIterator = oak.keysIterator(); - return Iterators.transform(keysIterator, key -> key); + try (OakBufferView bufferView = oak.createBufferView()) { + + OakIterator> iterator = bufferView.entriesIterator(); + return Iterators.transform(iterator, entry -> + new OakIncrementalIndexRow(entry.getKey(), dimensionDescsList, entry.getValue())); + } }; } @@ -622,72 +628,61 @@ public void close() protected float getMetricFloatValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - //TODO YONIGO - all these get functions are bad!. we should somehow put the aggregator bytebuffer - // in the incrementalindexrow to avoid and additional search in oak. - Function, Float> transformer = entry -> { - ByteBuffer serializedValue = entry.getValue(); + Function transformer = serializedValue -> { BufferAggregator agg = aggsManager.getAggs()[aggIndex]; return agg.getFloat(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); }; - try (OakTransformView transformView = oak.createTransformView(transformer)) { - return transformView.get(incrementalIndexRow); - } + OakRBuffer rBuffer = ((OakIncrementalIndexRow) incrementalIndexRow).getAggregations(); + return rBuffer.transform(transformer); } protected long getMetricLongValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - Function, Long> transformer = entry -> { - ByteBuffer serializedValue = entry.getValue(); + Function transformer = serializedValue -> { BufferAggregator agg = aggsManager.getAggs()[aggIndex]; return agg.getLong(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); }; - try (OakTransformView transformView = oak.createTransformView(transformer)) { - return transformView.get(incrementalIndexRow); - } + OakRBuffer rBuffer = ((OakIncrementalIndexRow) incrementalIndexRow).getAggregations(); + return rBuffer.transform(transformer); } protected Object getMetricObjectValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - Function, Object> transformer = entry -> { - ByteBuffer serializedValue = entry.getValue(); + Function transformer = serializedValue -> { BufferAggregator agg = aggsManager.getAggs()[aggIndex]; return agg.get(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); }; - try (OakTransformView transformView = oak.createTransformView(transformer)) { - return transformView.get(incrementalIndexRow); - } + OakRBuffer rBuffer = ((OakIncrementalIndexRow) incrementalIndexRow).getAggregations(); + return rBuffer.transform(transformer); } protected double getMetricDoubleValue(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - Function, Double> transformer = entry -> { - ByteBuffer serializedValue = entry.getValue(); + Function transformer = serializedValue -> { BufferAggregator agg = aggsManager.getAggs()[aggIndex]; return agg.getDouble(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); }; - try (OakTransformView transformView = oak.createTransformView(transformer)) { - return transformView.get(incrementalIndexRow); - } + + OakRBuffer rBuffer = ((OakIncrementalIndexRow) incrementalIndexRow).getAggregations(); + return rBuffer.transform(transformer); } protected boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggIndex) { - Function, Boolean> transformer = entry -> { - ByteBuffer serializedValue = entry.getValue(); + Function transformer = serializedValue -> { BufferAggregator agg = aggsManager.getAggs()[aggIndex]; return agg.isNull(serializedValue, serializedValue.position() + aggsManager.aggOffsetInBuffer[aggIndex]); }; - try (OakTransformView transformView = oak.createTransformView(transformer)) { - return transformView.get(incrementalIndexRow); - } + OakRBuffer rBuffer = ((OakIncrementalIndexRow) incrementalIndexRow).getAggregations(); + return rBuffer.transform(transformer); } } } 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 index 66a344dbba5e..d2f213bc4d71 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.incremental; +import com.oath.oak.OakRBuffer; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex.DimensionDesc; @@ -30,20 +31,29 @@ //wrapper class for bytebuffers public class OakIncrementalIndexRow extends IncrementalIndexRow { - private final ByteBuffer byteBuffer; + private final ByteBuffer dimentions; + private final OakRBuffer aggregations; private Integer dimsLength; - public OakIncrementalIndexRow(ByteBuffer byteBuffer, List dimensionDescsList) + public OakIncrementalIndexRow(ByteBuffer dimentions, + List dimensionDescsList, + OakRBuffer aggregations) { super(0, dimensionDescsList); - this.byteBuffer = byteBuffer; + this.dimentions = dimentions; this.dimsLength = null; // lazy initialization + this.aggregations = aggregations; + } + + public OakRBuffer getAggregations() + { + return aggregations; } @Override public long getTimestamp() { - return byteBuffer.getLong(byteBuffer.position() + OakIncrementalIndex.TIME_STAMP_INDEX); + return dimentions.getLong(dimentions.position() + OakIncrementalIndex.TIME_STAMP_INDEX); } @Override @@ -51,7 +61,7 @@ public int getDimsLength() { // Read length only once if (dimsLength == null) { - dimsLength = byteBuffer.getInt(byteBuffer.position() + OakIncrementalIndex.DIMS_LENGTH_INDEX); + dimsLength = dimentions.getInt(dimentions.position() + OakIncrementalIndex.DIMS_LENGTH_INDEX); } return dimsLength; } @@ -68,7 +78,7 @@ public Object getDim(int dimIndex) @Override public int getRowIndex() { - return byteBuffer.getInt(byteBuffer.position() + OakIncrementalIndex.ROW_INDEX_INDEX); + return dimentions.getInt(dimentions.position() + OakIncrementalIndex.ROW_INDEX_INDEX); } @Override @@ -98,7 +108,7 @@ public long estimateBytesInMemory() int dimType = getDimType(dimIndex); if (dimType == ValueType.STRING.ordinal()) { int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - int arraySize = byteBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + int arraySize = dimentions.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); sizeInBytes += (arraySize * Integer.BYTES); } } @@ -115,7 +125,7 @@ private int getDimIndexInBuffer(int dimIndex) if (dimIndex >= getDimsLength()) { return OakIncrementalIndex.NO_DIM; } - return byteBuffer.position() + OakIncrementalIndex.DIMS_INDEX + dimIndex * OakIncrementalIndex.ALLOC_PER_DIM; + return dimentions.position() + OakIncrementalIndex.DIMS_INDEX + dimIndex * OakIncrementalIndex.ALLOC_PER_DIM; } private Object getDimValue(int dimIndex) @@ -125,22 +135,22 @@ private Object getDimValue(int dimIndex) return null; } int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - int dimType = byteBuffer.getInt(dimIndexInBuffer); + int dimType = dimentions.getInt(dimIndexInBuffer); if (dimType == OakIncrementalIndex.NO_DIM) { return null; } else if (dimType == ValueType.DOUBLE.ordinal()) { - dimObject = byteBuffer.getDouble(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + dimObject = dimentions.getDouble(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); } else if (dimType == ValueType.FLOAT.ordinal()) { - dimObject = byteBuffer.getFloat(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + dimObject = dimentions.getFloat(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); } else if (dimType == ValueType.LONG.ordinal()) { - dimObject = byteBuffer.getLong(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + dimObject = dimentions.getLong(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); } else if (dimType == ValueType.STRING.ordinal()) { - int arrayIndexOffset = byteBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_INDEX_OFFSET); - int arrayIndex = byteBuffer.position() + arrayIndexOffset; - int arraySize = byteBuffer.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + int arrayIndexOffset = dimentions.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_INDEX_OFFSET); + int arrayIndex = dimentions.position() + arrayIndexOffset; + int arraySize = dimentions.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); int[] array = new int[arraySize]; for (int i = 0; i < arraySize; i++) { - array[i] = byteBuffer.getInt(arrayIndex); + array[i] = dimentions.getInt(arrayIndex); arrayIndex += Integer.BYTES; } dimObject = array; @@ -155,6 +165,6 @@ private int getDimType(int dimIndex) return OakIncrementalIndex.NO_DIM; } int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - return byteBuffer.getInt(dimIndexInBuffer); + return dimentions.getInt(dimIndexInBuffer); } } From b32ada20820f793203371c1e66d3e72228de01ba Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Sun, 17 Mar 2019 14:34:36 +0200 Subject: [PATCH 03/15] Restore defaults maxDirectMemory in pom --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2d9e5d4ce6bc..ba9d71304587 100644 --- a/pom.xml +++ b/pom.xml @@ -1220,7 +1220,7 @@ -Xmx1500m - -XX:MaxDirectMemorySize=32g + -XX:MaxDirectMemorySize=512m -Duser.language=en -Duser.GroupByQueryRunnerTest.javacountry=US -Dfile.encoding=UTF-8 From 8416ec22fb911de1ae5c2bf88221b17027362cf8 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Tue, 19 Mar 2019 15:29:30 +0200 Subject: [PATCH 04/15] rebase with master --- .../druid/segment/incremental/OakIncrementalIndex.java | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) 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 index 873b1d7d6505..7079a85b590a 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java @@ -51,7 +51,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; import java.util.function.Consumer; import java.util.function.Function; @@ -166,18 +165,15 @@ public void close() { facts.close(); } + @Override - protected AddToFactsResult addToFacts(AggregatorFactory[] metrics, - boolean deserializeComplexMetrics, - boolean reportParseExceptions, - InputRow row, - AtomicInteger numEntries, - AtomicLong sizeInBytes, + protected AddToFactsResult addToFacts(InputRow row, IncrementalIndexRow key, ThreadLocal rowContainer, Supplier rowSupplier, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException { + final AtomicInteger numEntries = getNumEntries(); return facts.addToOak(row, numEntries, key, rowContainer, skipMaxRowsInMemoryCheck); } From c6e198f94facb2248e5060ff18050b96a34920c7 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Sun, 31 Mar 2019 20:54:56 +0300 Subject: [PATCH 05/15] 1. Update to OakMap new API. 2. Fix addToFacts concurrent bug. 3. Add multithread insert benchmark --- .../indexing/IndexIngestionBenchmark.java | 4 - .../IndexIngestionMultithreadedBenchmark.java | 158 ++++++++++++++++++ .../druid/benchmark/query/TopNBenchmark.java | 2 +- .../incremental/OakIncrementalIndex.java | 101 +++++------ 4 files changed, 202 insertions(+), 63 deletions(-) create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionMultithreadedBenchmark.java 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 badc709d1505..1161f4afda2d 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 @@ -43,10 +43,6 @@ 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; 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..6ab4fc80da01 --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionMultithreadedBenchmark.java @@ -0,0 +1,158 @@ +/* + * 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.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.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +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 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) +public class IndexIngestionMultithreadedBenchmark +{ + @Param({"75000"}) + private int rowsPerSegment; + + @Param({"basic"}) + private String schema; + + @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; + + private IncrementalIndex incIndex; + private ArrayList rows; + private BenchmarkSchemaInfo schemaInfo; + AtomicInteger threadIdAllocator = new AtomicInteger(0); + + @Setup + public void setup() + { + 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(), + rowsPerSegment + ); + + for (int i = 0; i < rowsPerSegment; i++) { + InputRow row = gen.nextRow(); + if (i % 10000 == 0) { + log.info(i + " rows generated."); + } + + rows.add(row); + } + } + + @State(Scope.Thread) + public static class ThreadState + { + int id = 0; + + @Setup + public void setup(IndexIngestionMultithreadedBenchmark globalState) + { + id = globalState.threadIdAllocator.getAndIncrement(); + } + } + + @Setup(Level.Invocation) + public void setup2() + { + incIndex = makeIncIndex(); + } + + @TearDown(Level.Iteration) + public void tearDown() + { + incIndex.close(); + } + + private IncrementalIndex makeIncIndex() + { + IncrementalIndex.Builder builder = new IncrementalIndex.Builder() + .setIndexSchema( + new IncrementalIndexSchema.Builder() + .withMetrics(schemaInfo.getAggsArray()) + .withRollup(rollup) + .build() + ) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment * 2); + switch (indexType) { + case "onheap": + return builder.buildOnheap(); + case "oak": + return builder.buildOak(); + } + return null; + } + + @Benchmark + @BenchmarkMode(Mode.SingleShotTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void addRows(Blackhole blackhole, ThreadState threadState) throws Exception + { + int threads = threadIdAllocator.get(); + for (int i = threadState.id; i < rowsPerSegment; i += threads) { + InputRow row = rows.get(i); + int rv = incIndex.add(row).getRowCount(); + blackhole.consume(rv); + } + } +} 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 693eeb12b5a0..e1a45daf2ca5 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 @@ -293,7 +293,7 @@ public void setup() throws IOException public void tearDown() throws IOException { FileUtils.deleteDirectory(tmpDir); - incIndexes.forEach(index->index.close()); + incIndexes.forEach(index -> index.close()); } private IncrementalIndex makeIncIndex() 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 index 7079a85b590a..7a44fc4c6a4a 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java @@ -23,12 +23,9 @@ import com.google.common.base.Supplier; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; -import com.oath.oak.OakBufferView; -import com.oath.oak.OakIterator; import com.oath.oak.OakMap; import com.oath.oak.OakMapBuilder; import com.oath.oak.OakRBuffer; -import com.oath.oak.OakTransformView; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -51,12 +48,11 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.Function; - /** */ public class OakIncrementalIndex extends IncrementalIndex @@ -174,7 +170,8 @@ protected AddToFactsResult addToFacts(InputRow row, boolean skipMaxRowsInMemoryCheck) throws IndexSizeExceededException { final AtomicInteger numEntries = getNumEntries(); - return facts.addToOak(row, numEntries, key, rowContainer, skipMaxRowsInMemoryCheck); + final AtomicLong bytesInMemory = getBytesInMemory(); + return facts.addToOak(row, numEntries, bytesInMemory, key, rowContainer, skipMaxRowsInMemoryCheck); } @Override @@ -232,9 +229,9 @@ protected boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggOffset) public Iterable iterableWithPostAggregations(final List postAggs, final boolean descending) { //TODO YONIGO - rewrite this function. maybe return an unserialized row? - Function, Row> transformer = entry -> { + Function, Row> transformer = entry -> { ByteBuffer serializedKey = entry.getKey(); - ByteBuffer serializedValue = entry.getValue(); + OakRBuffer serializedValue = entry.getValue(); long timeStamp = OakIncrementalIndex.getTimestamp(serializedKey); int dimsLength = OakIncrementalIndex.getDimsLength(serializedKey); Map theVals = Maps.newLinkedHashMap(); @@ -257,16 +254,16 @@ public Iterable iterableWithPostAggregations(final List pos 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])); + BufferAggregator agg = aggs[i]; + int aggOffsetInBuffer = aggsManager.aggOffsetInBuffer[i]; + Object theVal = serializedValue.transform(bb -> agg.get(bb, aggOffsetInBuffer)); + theVals.put(aggsManager.metrics[i].getName(), theVal); } return new MapBasedRow(timeStamp, theVals); }; - return () -> { - OakIterator iterator = facts.transformIterator(descending, transformer); - return Iterators.transform(iterator, row -> row); - }; + return () -> facts.transformIterator(descending, transformer); } //static methods @@ -359,7 +356,6 @@ static Object getDimValue(ByteBuffer buff, int dimIndex, int dimsLength) static class AggsManager { private final AggregatorFactory[] metrics; - private final ReentrantLock[] aggLocks; private volatile Map selectors; private final boolean reportParseExceptions; @@ -381,10 +377,6 @@ public AggsManager(BufferAggregator[] aggs, this.aggOffsetInBuffer = aggOffsetInBuffer; this.aggsTotalSize = aggsTotalSize; this.metrics = metrics; - this.aggLocks = new ReentrantLock[this.aggs.length]; - for (int i = 0; i < this.aggLocks.length; i++) { - this.aggLocks[i] = new ReentrantLock(true); - } this.reportParseExceptions = reportParseExceptions; } @@ -392,19 +384,21 @@ public void initValue(ByteBuffer byteBuffer, InputRow row, ThreadLocal rowContainer) { - if (metrics.length > 0 && aggs[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]; - aggLocks[i].lock(); - if (aggs[i] == null) { - aggs[i] = agg.factorizeBuffered(selectors.get(agg.getName())); + if (metrics.length > 0 && aggs[aggs.length - 1] == null) { + synchronized (this) { + if (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]; + if (aggs[i] == null) { + aggs[i] = agg.factorizeBuffered(selectors.get(agg.getName())); + } + } + rowContainer.set(null); } - aggLocks[i].unlock(); } - rowContainer.set(null); } for (int i = 0; i < metrics.length; i++) { @@ -458,7 +452,6 @@ private static class OakFactsHolder implements FactsHolder private final long minTimestamp; private final List dimensionDescsList; - private final boolean rollup; private final int maxRowCount; private final AtomicInteger rowIndexGenerator; @@ -479,20 +472,16 @@ public OakFactsHolder(IncrementalIndexSchema incrementalIndexSchema, oak = builder.build(); this.minTimestamp = incrementalIndexSchema.getMinTimestamp(); this.dimensionDescsList = dimensionDescsList; - this.rollup = rollup; this.maxRowCount = maxRowCount; this.rowIndexGenerator = new AtomicInteger(0); this.aggsManager = aggsManager; } - public OakIterator transformIterator(boolean descending, Function, Row> transformer) + public Iterator transformIterator(boolean descending, Function, Row> transformer) { - try (OakMap tmpOakMap = descending ? oak.descendingMap() : oak; - OakTransformView transformView = tmpOakMap.createTransformView(transformer)) { - OakIterator valuesIterator = transformView.entriesIterator(); - return valuesIterator; - } + OakMap tmpOakMap = descending ? oak.descendingMap() : oak; + return tmpOakMap.zc().entrySet().stream().map(transformer).iterator(); } @@ -510,13 +499,13 @@ public int getPriorIndex(IncrementalIndexRow key) @Override public long getMinTimeMillis() { - return oak.getMinKey().getTimestamp(); + return oak.firstKey().getTimestamp(); } @Override public long getMaxTimeMillis() { - return oak.getMaxKey().getTimestamp(); + return oak.lastKey().getTimestamp(); } @Override @@ -535,12 +524,15 @@ public Iterable timeRangeIterable(boolean descending, long return () -> { IncrementalIndexRow from = new IncrementalIndexRow(timeStart, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); IncrementalIndexRow to = new IncrementalIndexRow(timeEnd, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); - try (OakMap subMap = oak.subMap(from, true, to, false, descending); - OakBufferView bufferView = subMap.createBufferView()) { - OakIterator> iterator = bufferView.entriesIterator(); - return Iterators.transform(iterator, entry -> - new OakIncrementalIndexRow(entry.getKey(), dimensionDescsList, entry.getValue())); + try (OakMap subMap = oak.subMap(from, true, to, false, descending)) { + Iterator> iterator = subMap + .zc() + .entrySet() + .iterator(); + //TODO YONIGO - what sorcery is this transform function doing to allow this?! + return Iterators.transform(iterator, + entry -> new OakIncrementalIndexRow(entry.getKey(), dimensionDescsList, entry.getValue())); } }; } @@ -548,14 +540,8 @@ public Iterable timeRangeIterable(boolean descending, long @Override public Iterable keySet() { - return () -> { - try (OakBufferView bufferView = oak.createBufferView()) { - - OakIterator> iterator = bufferView.entriesIterator(); - return Iterators.transform(iterator, entry -> - new OakIncrementalIndexRow(entry.getKey(), dimensionDescsList, entry.getValue())); - } - }; + return () -> Iterators.transform(oak.zc().entrySet().iterator(), entry -> + new OakIncrementalIndexRow(entry.getKey(), dimensionDescsList, entry.getValue())); } @Override @@ -574,14 +560,13 @@ public int putIfAbsent(IncrementalIndexRow key, int rowIndex) @Override public void clear() { - //TODO YONIGO - add clear to oak - //oak.clear(); + oak.clear(); } private AddToFactsResult addToOak( InputRow row, AtomicInteger numEntries, - IncrementalIndexRow incrementalIndexRow, + AtomicLong bytesInMemory, IncrementalIndexRow incrementalIndexRow, ThreadLocal rowContainer, boolean skipMaxRowsInMemoryCheck ) throws IndexSizeExceededException @@ -589,12 +574,12 @@ private AddToFactsResult addToOak( Consumer computer = buffer -> aggsManager.aggregate(row, rowContainer, buffer); incrementalIndexRow.setRowIndex(rowIndexGenerator.getAndIncrement()); - boolean added = oak.putIfAbsentComputeIfPresent(incrementalIndexRow, row, computer); + boolean added = oak.zc().putIfAbsentComputeIfPresent(incrementalIndexRow, row, computer); if (added) { numEntries.incrementAndGet(); } - //TODO YONIGO - we will continue to add and throw exceptions. + //TODO YONIGO - we will continue to add and throw exceptions. Cannot check before because of rollup. if ((numEntries.get() > maxRowCount) //TODO YONIGO: || sizeInBytes.get() >= maxBytesInMemory && !skipMaxRowsInMemoryCheck) { throw new IndexSizeExceededException( @@ -603,7 +588,7 @@ private AddToFactsResult addToOak( ); } - return new AddToFactsResult(oak.entries(), 0, new ArrayList<>()); + return new AddToFactsResult(oak.size(), 0, new ArrayList<>()); } public int getMaxRowCount() From f2a5cd657cff27b07fd7569dfc9274ba423876fd Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Sun, 31 Mar 2019 22:28:34 +0300 Subject: [PATCH 06/15] PR code review fixes --- .../segment/data/ArrayBasedIndexedInts.java | 8 -- .../incremental/IncrementalIndexRow.java | 2 +- .../incremental/OakIncrementalIndex.java | 105 +-------------- .../incremental/OakIncrementalIndexRow.java | 43 +++--- .../segment/incremental/OakKeySerializer.java | 34 ++--- .../incremental/OakKeysComparator.java | 46 +++---- .../druid/segment/incremental/OakUtils.java | 127 ++++++++++++++++++ 7 files changed, 192 insertions(+), 173 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java 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 1f4970e53ccb..2c48674c46bf 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 @@ -73,14 +73,6 @@ 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/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index f7b497baf1d3..2ccff4385f0c 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 @@ -36,7 +36,7 @@ public class IncrementalIndexRow final long timestamp; final Object[] dims; - protected final List dimensionDescsList; + private final List dimensionDescsList; /** * rowIndex is not checked in {@link #equals} and {@link #hashCode} on purpose. IncrementalIndexRow acts as a Map key 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 index 7a44fc4c6a4a..fc5662ea9630 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java @@ -38,8 +38,6 @@ import org.apache.druid.segment.ColumnSelectorFactory; 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.reflect.generics.reflectiveObjects.NotImplementedException; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -58,17 +56,7 @@ public class OakIncrementalIndex extends IncrementalIndex { - 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; + private final OakFactsHolder facts; @@ -232,11 +220,11 @@ public Iterable iterableWithPostAggregations(final List pos Function, Row> transformer = entry -> { ByteBuffer serializedKey = entry.getKey(); OakRBuffer serializedValue = entry.getValue(); - long timeStamp = OakIncrementalIndex.getTimestamp(serializedKey); - int dimsLength = OakIncrementalIndex.getDimsLength(serializedKey); + long timeStamp = OakUtils.getTimestamp(serializedKey); + int dimsLength = OakUtils.getDimsLength(serializedKey); Map theVals = Maps.newLinkedHashMap(); for (int i = 0; i < dimsLength; ++i) { - Object dim = OakIncrementalIndex.getDimValue(serializedKey, i); + Object dim = OakUtils.getDimValue(serializedKey, i); DimensionDesc dimensionDesc = dimensionDescsList.get(i); if (dimensionDesc == null) { continue; @@ -266,91 +254,6 @@ public Iterable iterableWithPostAggregations(final List pos return () -> facts.transformIterator(descending, transformer); } -//static methods - - 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 long getTimestamp(ByteBuffer buff) - { - return buff.getLong(buff.position() + TIME_STAMP_INDEX); - } - - static int getRowIndex(ByteBuffer buff) - { - return buff.getInt(buff.position() + ROW_INDEX_INDEX); - } - - 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 Object getDimValue(ByteBuffer buff, int dimIndex) - { - int dimsLength = getDimsLength(buff); - return getDimValue(buff, dimIndex, dimsLength); - } - - 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) - { - 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 class AggsManager 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 index d2f213bc4d71..28d808803319 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.incremental; - import com.oath.oak.OakRBuffer; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex.DimensionDesc; @@ -27,11 +26,9 @@ import java.nio.ByteBuffer; import java.util.List; - -//wrapper class for bytebuffers public class OakIncrementalIndexRow extends IncrementalIndexRow { - private final ByteBuffer dimentions; + private final ByteBuffer dimensions; private final OakRBuffer aggregations; private Integer dimsLength; @@ -40,7 +37,7 @@ public OakIncrementalIndexRow(ByteBuffer dimentions, OakRBuffer aggregations) { super(0, dimensionDescsList); - this.dimentions = dimentions; + this.dimensions = dimentions; this.dimsLength = null; // lazy initialization this.aggregations = aggregations; } @@ -53,7 +50,7 @@ public OakRBuffer getAggregations() @Override public long getTimestamp() { - return dimentions.getLong(dimentions.position() + OakIncrementalIndex.TIME_STAMP_INDEX); + return dimensions.getLong(dimensions.position() + OakUtils.TIME_STAMP_INDEX); } @Override @@ -61,7 +58,7 @@ public int getDimsLength() { // Read length only once if (dimsLength == null) { - dimsLength = dimentions.getInt(dimentions.position() + OakIncrementalIndex.DIMS_LENGTH_INDEX); + dimsLength = dimensions.getInt(dimensions.position() + OakUtils.DIMS_LENGTH_INDEX); } return dimsLength; } @@ -78,7 +75,7 @@ public Object getDim(int dimIndex) @Override public int getRowIndex() { - return dimentions.getInt(dimentions.position() + OakIncrementalIndex.ROW_INDEX_INDEX); + return dimensions.getInt(dimensions.position() + OakUtils.ROW_INDEX_INDEX); } @Override @@ -104,11 +101,11 @@ public long estimateBytesInMemory() long sizeInBytes = Long.BYTES + 2 * Integer.BYTES; for (int dimIndex = 0; dimIndex < getDimsLength(); dimIndex++) { - sizeInBytes += OakIncrementalIndex.ALLOC_PER_DIM; + sizeInBytes += OakUtils.ALLOC_PER_DIM; int dimType = getDimType(dimIndex); if (dimType == ValueType.STRING.ordinal()) { int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - int arraySize = dimentions.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + int arraySize = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_LENGTH_OFFSET); sizeInBytes += (arraySize * Integer.BYTES); } } @@ -123,9 +120,9 @@ public long estimateBytesInMemory() private int getDimIndexInBuffer(int dimIndex) { if (dimIndex >= getDimsLength()) { - return OakIncrementalIndex.NO_DIM; + return OakUtils.NO_DIM; } - return dimentions.position() + OakIncrementalIndex.DIMS_INDEX + dimIndex * OakIncrementalIndex.ALLOC_PER_DIM; + return dimensions.position() + OakUtils.DIMS_INDEX + dimIndex * OakUtils.ALLOC_PER_DIM; } private Object getDimValue(int dimIndex) @@ -135,22 +132,22 @@ private Object getDimValue(int dimIndex) return null; } int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - int dimType = dimentions.getInt(dimIndexInBuffer); - if (dimType == OakIncrementalIndex.NO_DIM) { + int dimType = dimensions.getInt(dimIndexInBuffer); + if (dimType == OakUtils.NO_DIM) { return null; } else if (dimType == ValueType.DOUBLE.ordinal()) { - dimObject = dimentions.getDouble(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + dimObject = dimensions.getDouble(dimIndexInBuffer + OakUtils.DATA_OFFSET); } else if (dimType == ValueType.FLOAT.ordinal()) { - dimObject = dimentions.getFloat(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + dimObject = dimensions.getFloat(dimIndexInBuffer + OakUtils.DATA_OFFSET); } else if (dimType == ValueType.LONG.ordinal()) { - dimObject = dimentions.getLong(dimIndexInBuffer + OakIncrementalIndex.DATA_OFFSET); + dimObject = dimensions.getLong(dimIndexInBuffer + OakUtils.DATA_OFFSET); } else if (dimType == ValueType.STRING.ordinal()) { - int arrayIndexOffset = dimentions.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_INDEX_OFFSET); - int arrayIndex = dimentions.position() + arrayIndexOffset; - int arraySize = dimentions.getInt(dimIndexInBuffer + OakIncrementalIndex.ARRAY_LENGTH_OFFSET); + int arrayIndexOffset = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_INDEX_OFFSET); + int arrayIndex = dimensions.position() + arrayIndexOffset; + int arraySize = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_LENGTH_OFFSET); int[] array = new int[arraySize]; for (int i = 0; i < arraySize; i++) { - array[i] = dimentions.getInt(arrayIndex); + array[i] = dimensions.getInt(arrayIndex); arrayIndex += Integer.BYTES; } dimObject = array; @@ -162,9 +159,9 @@ private Object getDimValue(int dimIndex) private int getDimType(int dimIndex) { if (dimIndex >= getDimsLength()) { - return OakIncrementalIndex.NO_DIM; + return OakUtils.NO_DIM; } int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - return dimentions.getInt(dimIndexInBuffer); + return dimensions.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 index f33c81d950b5..6cd964a25c67 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java @@ -45,29 +45,29 @@ public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBu // calculating buffer indexes for writing the key data int buffIndex = byteBuffer.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 + int timeStampIndex = buffIndex + OakUtils.TIME_STAMP_INDEX; // the timestamp index + int dimsLengthIndex = buffIndex + OakUtils.DIMS_LENGTH_INDEX; // the dims array length index + int rowIndexIndex = buffIndex + OakUtils.ROW_INDEX_INDEX; // the rowIndex index + int dimsIndex = buffIndex + OakUtils.DIMS_INDEX; // the dims array index + int dimCapacity = OakUtils.ALLOC_PER_DIM; // the number of bytes required // per dim - int noDim = OakIncrementalIndex.NO_DIM; // for mentioning that + int noDim = OakUtils.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 + int valueTypeOffset = OakUtils.VALUE_TYPE_OFFSET; // offset from the dimIndex + int dataOffset = OakUtils.DATA_OFFSET; // for non-STRING dims + int arrayIndexOffset = OakUtils.ARRAY_INDEX_OFFSET; // for STRING dims + int arrayLengthOffset = OakUtils.ARRAY_LENGTH_OFFSET; // for STRING dims byteBuffer.putLong(timeStampIndex, timestamp); byteBuffer.putInt(dimsLengthIndex, dimsLength); byteBuffer.putInt(rowIndexIndex, rowIndex); for (int i = 0; i < dimsLength; i++) { - ValueType valueType = OakIncrementalIndex.getDimValueType(i, dimensionDescsList); + ValueType valueType = OakUtils.getDimValueType(i, dimensionDescsList); if (valueType == null || incrementalIndexRow.getDim(i) == null) { byteBuffer.putInt(dimsIndex, noDim); } else { @@ -105,12 +105,12 @@ public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBu @Override public IncrementalIndexRow deserialize(ByteBuffer byteBuffer) { - long timeStamp = OakIncrementalIndex.getTimestamp(byteBuffer); - int dimsLength = OakIncrementalIndex.getDimsLength(byteBuffer); - int rowIndex = OakIncrementalIndex.getRowIndex(byteBuffer); + long timeStamp = OakUtils.getTimestamp(byteBuffer); + int dimsLength = OakUtils.getDimsLength(byteBuffer); + int rowIndex = OakUtils.getRowIndex(byteBuffer); Object[] dims = new Object[dimsLength]; for (int dimIndex = 0; dimIndex < dimsLength; dimIndex++) { - Object dim = OakIncrementalIndex.getDimValue(byteBuffer, dimIndex, dimsLength); + Object dim = OakUtils.getDimValue(byteBuffer, dimIndex, dimsLength); dims[dimIndex] = dim; } return new IncrementalIndexRow(timeStamp, dims, dimensionDescsList, rowIndex); @@ -133,7 +133,7 @@ public int calculateSize(IncrementalIndexRow incrementalIndexRow) if (dim == null) { continue; } - if (OakIncrementalIndex.getDimValueType(i, dimensionDescsList) == ValueType.STRING) { + if (OakUtils.getDimValueType(i, dimensionDescsList) == ValueType.STRING) { sumOfArrayLengths += ((int[]) dim).length; } } @@ -144,7 +144,7 @@ public int calculateSize(IncrementalIndexRow incrementalIndexRow) // 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 dimCapacity = OakUtils.ALLOC_PER_DIM; int allocSize = Long.BYTES + 2 * Integer.BYTES + dimCapacity * incrementalIndexRow.getDimsLength() + 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 index 47f73fa7c978..6a511f1f6f7f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java @@ -93,31 +93,31 @@ public int compareKeys(IncrementalIndexRow lhs, IncrementalIndexRow rhs) @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 retVal = Longs.compare(OakUtils.getTimestamp(lhs), OakUtils.getTimestamp(rhs)); + int lhsDimsLength = OakUtils.getDimsLength(lhs); + int rhsDimsLength = OakUtils.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)); + int lhsType = lhs.getInt(OakUtils.getDimIndexInBuffer(lhs, lhsDimsLength, dimIndex)); + int rhsType = rhs.getInt(OakUtils.getDimIndexInBuffer(rhs, rhsDimsLength, dimIndex)); - if (lhsType == OakIncrementalIndex.NO_DIM) { - if (rhsType == OakIncrementalIndex.NO_DIM) { + if (lhsType == OakUtils.NO_DIM) { + if (rhsType == OakUtils.NO_DIM) { ++dimIndex; continue; } return -1; } - if (rhsType == OakIncrementalIndex.NO_DIM) { + if (rhsType == OakUtils.NO_DIM) { return 1; } final DimensionIndexer indexer = dimensionDescsList.get(dimIndex).getIndexer(); - Object lhsObject = OakIncrementalIndex.getDimValue(lhs, dimIndex); - Object rhsObject = OakIncrementalIndex.getDimValue(rhs, dimIndex); + Object lhsObject = OakUtils.getDimValue(lhs, dimIndex); + Object rhsObject = OakUtils.getDimValue(rhs, dimIndex); retVal = indexer.compareUnsortedEncodedKeyComponents(lhsObject, rhsObject); ++dimIndex; } @@ -125,17 +125,17 @@ public int compareSerializedKeys(ByteBuffer lhs, ByteBuffer rhs) if (retVal == 0) { int lengthDiff = Ints.compare(lhsDimsLength, rhsDimsLength); if (lengthDiff == 0) { - return lastCompare(OakIncrementalIndex.getRowIndex(lhs), OakIncrementalIndex.getRowIndex(rhs)); + return lastCompare(OakUtils.getRowIndex(lhs), OakUtils.getRowIndex(rhs)); } if (lengthDiff > 0) { // lhs has bigger dims - if (OakIncrementalIndex.checkDimsAllNull(lhs, numComparisons)) { - return lastCompare(OakIncrementalIndex.getRowIndex(lhs), OakIncrementalIndex.getRowIndex(rhs)); + if (OakUtils.checkDimsAllNull(lhs, numComparisons)) { + return lastCompare(OakUtils.getRowIndex(lhs), OakUtils.getRowIndex(rhs)); } } else { // rhs has bigger dims - if (OakIncrementalIndex.checkDimsAllNull(rhs, numComparisons)) { - return lastCompare(OakIncrementalIndex.getRowIndex(lhs), OakIncrementalIndex.getRowIndex(rhs)); + if (OakUtils.checkDimsAllNull(rhs, numComparisons)) { + return lastCompare(OakUtils.getRowIndex(lhs), OakUtils.getRowIndex(rhs)); } } return lengthDiff; @@ -148,7 +148,7 @@ private int lastCompare(int lsIndex, int rsIndex) { if (!rollup || lsIndex == IncrementalIndexRow.EMPTY_ROW_INDEX || rsIndex == IncrementalIndexRow.EMPTY_ROW_INDEX) { // If we are not rollup then keys shouldnt collide. - // If on of the keys is EMPTY_ROW_INDEX this is a lower or upper bound key and must be compared. + // If one of the keys is EMPTY_ROW_INDEX this is a lower or upper bound key and must be compared. return lsIndex - rsIndex; } else { return 0; @@ -159,14 +159,14 @@ private int lastCompare(int lsIndex, int rsIndex) @Override public int compareSerializedKeyAndKey(ByteBuffer lhs, IncrementalIndexRow rhs) { - int retVal = Longs.compare(OakIncrementalIndex.getTimestamp(lhs), rhs.getTimestamp()); - int lhsDimsLength = OakIncrementalIndex.getDimsLength(lhs); + int retVal = Longs.compare(OakUtils.getTimestamp(lhs), rhs.getTimestamp()); + int lhsDimsLength = OakUtils.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 lhsIdxs = OakUtils.getDimValue(lhs, index); final Object rhsIdxs = rhs.getDim(index); if (lhsIdxs == null) { @@ -189,17 +189,17 @@ public int compareSerializedKeyAndKey(ByteBuffer lhs, IncrementalIndexRow rhs) if (retVal == 0) { int lengthDiff = Ints.compare(lhsDimsLength, rhsDimsLength); if (lengthDiff == 0) { - return lastCompare(OakIncrementalIndex.getRowIndex(lhs), rhs.getRowIndex()); + return lastCompare(OakUtils.getRowIndex(lhs), rhs.getRowIndex()); } if (lengthDiff > 0) { // lhs has bigger dims - if (OakIncrementalIndex.checkDimsAllNull(lhs, numComparisons)) { - return lastCompare(OakIncrementalIndex.getRowIndex(lhs), rhs.getRowIndex()); + if (OakUtils.checkDimsAllNull(lhs, numComparisons)) { + return lastCompare(OakUtils.getRowIndex(lhs), rhs.getRowIndex()); } } else { // rhs has bigger dims if (allNull(rhs, numComparisons)) { - return lastCompare(OakIncrementalIndex.getRowIndex(lhs), rhs.getRowIndex()); + return lastCompare(OakUtils.getRowIndex(lhs), rhs.getRowIndex()); } } return lengthDiff; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java new file mode 100644 index 000000000000..ac573aa69602 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java @@ -0,0 +1,127 @@ +/* + * 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.ColumnCapabilitiesImpl; +import org.apache.druid.segment.column.ValueType; +import java.nio.ByteBuffer; +import java.util.List; + +public final class OakUtils +{ + 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; + + private OakUtils() + { + } + + 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)) != NO_DIM) { + return false; + } + } + return true; + } + + static long getTimestamp(ByteBuffer buff) + { + return buff.getLong(buff.position() + TIME_STAMP_INDEX); + } + + static int getRowIndex(ByteBuffer buff) + { + return buff.getInt(buff.position() + ROW_INDEX_INDEX); + } + + static ValueType getDimValueType(int dimIndex, List dimensionDescsList) + { + IncrementalIndex.DimensionDesc dimensionDesc = dimensionDescsList.get(dimIndex); + if (dimensionDesc == null) { + return null; + } + ColumnCapabilitiesImpl capabilities = dimensionDesc.getCapabilities(); + if (capabilities == null) { + return null; + } + return capabilities.getType(); + } + + static Object getDimValue(ByteBuffer buff, int dimIndex) + { + int dimsLength = getDimsLength(buff); + return getDimValue(buff, dimIndex, dimsLength); + } + + 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) + { + Object dimObject = null; + if (dimIndex >= dimsLength) { + return null; + } + int dimIndexInBuffer = getDimIndexInBuffer(buff, dimsLength, dimIndex); + int dimType = buff.getInt(dimIndexInBuffer); + if (dimType == NO_DIM) { + return null; + } else if (dimType == ValueType.DOUBLE.ordinal()) { + dimObject = buff.getDouble(dimIndexInBuffer + DATA_OFFSET); + } else if (dimType == ValueType.FLOAT.ordinal()) { + dimObject = buff.getFloat(dimIndexInBuffer + DATA_OFFSET); + } else if (dimType == ValueType.LONG.ordinal()) { + dimObject = buff.getLong(dimIndexInBuffer + DATA_OFFSET); + } else if (dimType == ValueType.STRING.ordinal()) { + int arrayIndexOffset = buff.getInt(dimIndexInBuffer + ARRAY_INDEX_OFFSET); + int arrayIndex = buff.position() + arrayIndexOffset; + int arraySize = buff.getInt(dimIndexInBuffer + 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; + } +} From bda92f597b4ab9ba65969ef6bdc8f67ff36ec106 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Mon, 1 Apr 2019 12:22:53 +0300 Subject: [PATCH 07/15] 1. merge all keyCompare functions to one. 2. Clean code --- .../incremental/IncrementalIndexRow.java | 9 ++ .../incremental/OakIncrementalIndexRow.java | 50 ++------ .../segment/incremental/OakKeySerializer.java | 29 +++-- .../incremental/OakKeysComparator.java | 111 ++---------------- .../druid/segment/incremental/OakUtils.java | 26 ---- 5 files changed, 56 insertions(+), 169 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index 2ccff4385f0c..cf3256292edc 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 @@ -206,4 +206,13 @@ public int hashCode() } return hash; } + + boolean isDimNull(int index) + { + if (dims == null || index >= dims.length || dims[index] == null) { + return true; + } else { + return false; + } + } } 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 index 28d808803319..efd620b166c9 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -50,7 +50,7 @@ public OakRBuffer getAggregations() @Override public long getTimestamp() { - return dimensions.getLong(dimensions.position() + OakUtils.TIME_STAMP_INDEX); + return OakUtils.getTimestamp(dimensions); } @Override @@ -58,7 +58,7 @@ public int getDimsLength() { // Read length only once if (dimsLength == null) { - dimsLength = dimensions.getInt(dimensions.position() + OakUtils.DIMS_LENGTH_INDEX); + dimsLength = OakUtils.getDimsLength(dimensions); } return dimsLength; } @@ -75,7 +75,7 @@ public Object getDim(int dimIndex) @Override public int getRowIndex() { - return dimensions.getInt(dimensions.position() + OakUtils.ROW_INDEX_INDEX); + return OakUtils.getRowIndex(dimensions); } @Override @@ -113,47 +113,16 @@ public long estimateBytesInMemory() } - - /* ---------------- OakRBuffer utils -------------- */ private int getDimIndexInBuffer(int dimIndex) { - if (dimIndex >= getDimsLength()) { - return OakUtils.NO_DIM; - } - return dimensions.position() + OakUtils.DIMS_INDEX + dimIndex * OakUtils.ALLOC_PER_DIM; + return OakUtils.getDimIndexInBuffer(dimensions, getDimsLength(), dimIndex); } private Object getDimValue(int dimIndex) { - Object dimObject = null; - if (dimIndex >= getDimsLength()) { - return null; - } - int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - int dimType = dimensions.getInt(dimIndexInBuffer); - if (dimType == OakUtils.NO_DIM) { - return null; - } else if (dimType == ValueType.DOUBLE.ordinal()) { - dimObject = dimensions.getDouble(dimIndexInBuffer + OakUtils.DATA_OFFSET); - } else if (dimType == ValueType.FLOAT.ordinal()) { - dimObject = dimensions.getFloat(dimIndexInBuffer + OakUtils.DATA_OFFSET); - } else if (dimType == ValueType.LONG.ordinal()) { - dimObject = dimensions.getLong(dimIndexInBuffer + OakUtils.DATA_OFFSET); - } else if (dimType == ValueType.STRING.ordinal()) { - int arrayIndexOffset = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_INDEX_OFFSET); - int arrayIndex = dimensions.position() + arrayIndexOffset; - int arraySize = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_LENGTH_OFFSET); - int[] array = new int[arraySize]; - for (int i = 0; i < arraySize; i++) { - array[i] = dimensions.getInt(arrayIndex); - arrayIndex += Integer.BYTES; - } - dimObject = array; - } - - return dimObject; + return OakUtils.getDimValue(dimensions, dimIndex); } private int getDimType(int dimIndex) @@ -162,6 +131,13 @@ private int getDimType(int dimIndex) return OakUtils.NO_DIM; } int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); - return dimensions.getInt(dimIndexInBuffer); + return dimensions.getInt(dimIndexInBuffer + OakUtils.VALUE_TYPE_OFFSET); + } + + //Faster to check this way if dim is null instead of deserializing + @Override + boolean isDimNull(int index) + { + return dimensions.getInt(getDimIndexInBuffer(index)) == OakUtils.NO_DIM; } } 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 index 6cd964a25c67..85a791a0e756 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.incremental; import com.oath.oak.OakSerializer; +import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ValueType; import java.nio.ByteBuffer; import java.util.List; @@ -35,6 +36,22 @@ public OakKeySerializer(List dimensionDescsList) this.dimensionDescsList = dimensionDescsList; } + + private ValueType getDimValueType(int dimIndex, List dimensionDescsList) + { + IncrementalIndex.DimensionDesc dimensionDesc = dimensionDescsList.get(dimIndex); + if (dimensionDesc == null) { + return null; + } + ColumnCapabilitiesImpl capabilities = dimensionDesc.getCapabilities(); + if (capabilities == null) { + return null; + } + return capabilities.getType(); + } + + + @Override public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBuffer) { @@ -50,8 +67,6 @@ public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBu int rowIndexIndex = buffIndex + OakUtils.ROW_INDEX_INDEX; // the rowIndex index int dimsIndex = buffIndex + OakUtils.DIMS_INDEX; // the dims array index int dimCapacity = OakUtils.ALLOC_PER_DIM; // the number of bytes required - // per dim - int noDim = OakUtils.NO_DIM; // for mentioning that // a certain dim is null int dimsArraysIndex = dimsIndex + dimCapacity * dimsLength; // the index for // writing the int arrays @@ -67,9 +82,9 @@ public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBu byteBuffer.putInt(dimsLengthIndex, dimsLength); byteBuffer.putInt(rowIndexIndex, rowIndex); for (int i = 0; i < dimsLength; i++) { - ValueType valueType = OakUtils.getDimValueType(i, dimensionDescsList); + ValueType valueType = getDimValueType(i, dimensionDescsList); if (valueType == null || incrementalIndexRow.getDim(i) == null) { - byteBuffer.putInt(dimsIndex, noDim); + byteBuffer.putInt(dimsIndex, OakUtils.NO_DIM); } else { byteBuffer.putInt(dimsIndex + valueTypeOffset, valueType.ordinal()); switch (valueType) { @@ -93,7 +108,7 @@ public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBu dimsArrayOffset += (arr.length * Integer.BYTES); break; default: - byteBuffer.putInt(dimsIndex, noDim); + byteBuffer.putInt(dimsIndex, OakUtils.NO_DIM); } } @@ -119,7 +134,7 @@ public IncrementalIndexRow deserialize(ByteBuffer byteBuffer) @Override public int calculateSize(IncrementalIndexRow incrementalIndexRow) { - //TODO - YONIGO befrore this what == null is it correct now? + //TODO - YONIGO befrore this was == null is it correct now? if (incrementalIndexRow.getDimsLength() == 0) { return Long.BYTES + 2 * Integer.BYTES; } @@ -133,7 +148,7 @@ public int calculateSize(IncrementalIndexRow incrementalIndexRow) if (dim == null) { continue; } - if (OakUtils.getDimValueType(i, dimensionDescsList) == ValueType.STRING) { + if (getDimValueType(i, dimensionDescsList) == ValueType.STRING) { sumOfArrayLengths += ((int[]) dim).length; } } 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 index 6a511f1f6f7f..8be93272db03 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java @@ -40,8 +40,8 @@ public OakKeysComparator(List dimensionDescsList this.rollup = rollup; } - @Override - public int compareKeys(IncrementalIndexRow lhs, IncrementalIndexRow rhs) + + private int compare(IncrementalIndexRow lhs, IncrementalIndexRow rhs) { int retVal = Longs.compare(lhs.getTimestamp(), rhs.getTimestamp()); int lhsDimsLength = lhs.getDimsLength(); @@ -91,57 +91,17 @@ public int compareKeys(IncrementalIndexRow lhs, IncrementalIndexRow rhs) } @Override - public int compareSerializedKeys(ByteBuffer lhs, ByteBuffer rhs) + public int compareKeys(IncrementalIndexRow lhs, IncrementalIndexRow rhs) { - int retVal = Longs.compare(OakUtils.getTimestamp(lhs), OakUtils.getTimestamp(rhs)); - int lhsDimsLength = OakUtils.getDimsLength(lhs); - int rhsDimsLength = OakUtils.getDimsLength(rhs); - int numComparisons = Math.min(lhsDimsLength, rhsDimsLength); - - int dimIndex = 0; - while (retVal == 0 && dimIndex < numComparisons) { - int lhsType = lhs.getInt(OakUtils.getDimIndexInBuffer(lhs, lhsDimsLength, dimIndex)); - int rhsType = rhs.getInt(OakUtils.getDimIndexInBuffer(rhs, rhsDimsLength, dimIndex)); - - if (lhsType == OakUtils.NO_DIM) { - if (rhsType == OakUtils.NO_DIM) { - ++dimIndex; - continue; - } - return -1; - } - - if (rhsType == OakUtils.NO_DIM) { - return 1; - } - - final DimensionIndexer indexer = dimensionDescsList.get(dimIndex).getIndexer(); - Object lhsObject = OakUtils.getDimValue(lhs, dimIndex); - Object rhsObject = OakUtils.getDimValue(rhs, dimIndex); - retVal = indexer.compareUnsortedEncodedKeyComponents(lhsObject, rhsObject); - ++dimIndex; - } + return compare(lhs, rhs); + } - if (retVal == 0) { - int lengthDiff = Ints.compare(lhsDimsLength, rhsDimsLength); - if (lengthDiff == 0) { - return lastCompare(OakUtils.getRowIndex(lhs), OakUtils.getRowIndex(rhs)); - } - if (lengthDiff > 0) { - // lhs has bigger dims - if (OakUtils.checkDimsAllNull(lhs, numComparisons)) { - return lastCompare(OakUtils.getRowIndex(lhs), OakUtils.getRowIndex(rhs)); - } - } else { - // rhs has bigger dims - if (OakUtils.checkDimsAllNull(rhs, numComparisons)) { - return lastCompare(OakUtils.getRowIndex(lhs), OakUtils.getRowIndex(rhs)); - } - } - return lengthDiff; - } + @Override + public int compareSerializedKeys(ByteBuffer lhs, ByteBuffer rhs) + { + return compare(new OakIncrementalIndexRow(lhs, null, null), + new OakIncrementalIndexRow(rhs, null, null)); - return retVal; } private int lastCompare(int lsIndex, int rsIndex) @@ -159,63 +119,16 @@ private int lastCompare(int lsIndex, int rsIndex) @Override public int compareSerializedKeyAndKey(ByteBuffer lhs, IncrementalIndexRow rhs) { - int retVal = Longs.compare(OakUtils.getTimestamp(lhs), rhs.getTimestamp()); - int lhsDimsLength = OakUtils.getDimsLength(lhs); - int rhsDimsLength = rhs.getDimsLength(); - int numComparisons = Math.min(lhsDimsLength, rhsDimsLength); - - int index = 0; - while (retVal == 0 && index < numComparisons) { - final Object lhsIdxs = OakUtils.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) { - return lastCompare(OakUtils.getRowIndex(lhs), rhs.getRowIndex()); - } - if (lengthDiff > 0) { - // lhs has bigger dims - if (OakUtils.checkDimsAllNull(lhs, numComparisons)) { - return lastCompare(OakUtils.getRowIndex(lhs), rhs.getRowIndex()); - } - } else { - // rhs has bigger dims - if (allNull(rhs, numComparisons)) { - return lastCompare(OakUtils.getRowIndex(lhs), rhs.getRowIndex()); - } - } - return lengthDiff; - } - return retVal; + return compare(new OakIncrementalIndexRow(lhs, null, null), rhs); } private static boolean allNull(IncrementalIndexRow row, int startPosition) { for (int i = startPosition; i < row.getDimsLength(); i++) { - if (row.getDim(i) != null) { + if (!row.isDimNull(i)) { return false; } } return true; } - - } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java index ac573aa69602..919794f6ad78 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java @@ -19,10 +19,8 @@ package org.apache.druid.segment.incremental; -import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ValueType; import java.nio.ByteBuffer; -import java.util.List; public final class OakUtils { @@ -42,17 +40,6 @@ private OakUtils() { } - 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)) != NO_DIM) { - return false; - } - } - return true; - } - static long getTimestamp(ByteBuffer buff) { return buff.getLong(buff.position() + TIME_STAMP_INDEX); @@ -63,19 +50,6 @@ static int getRowIndex(ByteBuffer buff) return buff.getInt(buff.position() + ROW_INDEX_INDEX); } - static ValueType getDimValueType(int dimIndex, List dimensionDescsList) - { - IncrementalIndex.DimensionDesc dimensionDesc = dimensionDescsList.get(dimIndex); - if (dimensionDesc == null) { - return null; - } - ColumnCapabilitiesImpl capabilities = dimensionDesc.getCapabilities(); - if (capabilities == null) { - return null; - } - return capabilities.getType(); - } - static Object getDimValue(ByteBuffer buff, int dimIndex) { int dimsLength = getDimsLength(buff); From eebd410320a48a6bacc1d4f7d7e8dfa7f8bc1286 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Mon, 1 Apr 2019 15:49:00 +0300 Subject: [PATCH 08/15] Update to new Oak API changes --- .../incremental/OakIncrementalIndex.java | 11 ++-- .../incremental/OakIncrementalIndexRow.java | 26 ++++----- .../segment/incremental/OakKeySerializer.java | 2 +- .../incremental/OakKeysComparator.java | 49 +++++++++++++++-- .../druid/segment/incremental/OakUtils.java | 53 +++++++++++++++---- 5 files changed, 104 insertions(+), 37 deletions(-) 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 index fc5662ea9630..7f6b004e4fb7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndex.java @@ -26,6 +26,7 @@ import com.oath.oak.OakMap; import com.oath.oak.OakMapBuilder; import com.oath.oak.OakRBuffer; +import com.oath.oak.OakWBuffer; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; @@ -217,8 +218,8 @@ protected boolean isNull(IncrementalIndexRow incrementalIndexRow, int aggOffset) public Iterable iterableWithPostAggregations(final List postAggs, final boolean descending) { //TODO YONIGO - rewrite this function. maybe return an unserialized row? - Function, Row> transformer = entry -> { - ByteBuffer serializedKey = entry.getKey(); + Function, Row> transformer = entry -> { + OakRBuffer serializedKey = entry.getKey(); OakRBuffer serializedValue = entry.getValue(); long timeStamp = OakUtils.getTimestamp(serializedKey); int dimsLength = OakUtils.getDimsLength(serializedKey); @@ -381,7 +382,7 @@ public OakFactsHolder(IncrementalIndexSchema incrementalIndexSchema, } - public Iterator transformIterator(boolean descending, Function, Row> transformer) + public Iterator transformIterator(boolean descending, Function, Row> transformer) { OakMap tmpOakMap = descending ? oak.descendingMap() : oak; return tmpOakMap.zc().entrySet().stream().map(transformer).iterator(); @@ -429,7 +430,7 @@ public Iterable timeRangeIterable(boolean descending, long IncrementalIndexRow to = new IncrementalIndexRow(timeEnd, null, dimensionDescsList, IncrementalIndexRow.EMPTY_ROW_INDEX); try (OakMap subMap = oak.subMap(from, true, to, false, descending)) { - Iterator> iterator = subMap + Iterator> iterator = subMap .zc() .entrySet() .iterator(); @@ -475,7 +476,7 @@ private AddToFactsResult addToOak( ) throws IndexSizeExceededException { - Consumer computer = buffer -> aggsManager.aggregate(row, rowContainer, buffer); + Consumer computer = buffer -> aggsManager.aggregate(row, rowContainer, buffer.getByteBuffer()); incrementalIndexRow.setRowIndex(rowIndexGenerator.getAndIncrement()); boolean added = oak.zc().putIfAbsentComputeIfPresent(incrementalIndexRow, row, computer); if (added) { 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 index efd620b166c9..116b7dc177de 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -23,16 +23,15 @@ 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 { - private final ByteBuffer dimensions; + private final OakRBuffer dimensions; private final OakRBuffer aggregations; private Integer dimsLength; - public OakIncrementalIndexRow(ByteBuffer dimentions, + public OakIncrementalIndexRow(OakRBuffer dimentions, List dimensionDescsList, OakRBuffer aggregations) { @@ -69,7 +68,7 @@ public Object getDim(int dimIndex) if (dimIndex >= getDimsLength()) { return null; } - return getDimValue(dimIndex); + return OakUtils.getDimValue(dimensions, dimIndex); } @Override @@ -104,7 +103,7 @@ public long estimateBytesInMemory() sizeInBytes += OakUtils.ALLOC_PER_DIM; int dimType = getDimType(dimIndex); if (dimType == ValueType.STRING.ordinal()) { - int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + int dimIndexInBuffer = OakUtils.getDimIndexInBuffer(dimIndex); int arraySize = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_LENGTH_OFFSET); sizeInBytes += (arraySize * Integer.BYTES); } @@ -115,22 +114,12 @@ public long estimateBytesInMemory() /* ---------------- OakRBuffer utils -------------- */ - private int getDimIndexInBuffer(int dimIndex) - { - return OakUtils.getDimIndexInBuffer(dimensions, getDimsLength(), dimIndex); - } - - private Object getDimValue(int dimIndex) - { - return OakUtils.getDimValue(dimensions, dimIndex); - } - private int getDimType(int dimIndex) { if (dimIndex >= getDimsLength()) { return OakUtils.NO_DIM; } - int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + int dimIndexInBuffer = OakUtils.getDimIndexInBuffer(dimIndex); return dimensions.getInt(dimIndexInBuffer + OakUtils.VALUE_TYPE_OFFSET); } @@ -138,6 +127,9 @@ private int getDimType(int dimIndex) @Override boolean isDimNull(int index) { - return dimensions.getInt(getDimIndexInBuffer(index)) == OakUtils.NO_DIM; + if (index >= getDimsLength()) { + return true; + } + return dimensions.getInt(OakUtils.getDimIndexInBuffer(index)) == OakUtils.NO_DIM; } } 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 index 85a791a0e756..b265f9bc0c6e 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java @@ -125,7 +125,7 @@ public IncrementalIndexRow deserialize(ByteBuffer byteBuffer) int rowIndex = OakUtils.getRowIndex(byteBuffer); Object[] dims = new Object[dimsLength]; for (int dimIndex = 0; dimIndex < dimsLength; dimIndex++) { - Object dim = OakUtils.getDimValue(byteBuffer, dimIndex, dimsLength); + Object dim = OakUtils.getDimValue(byteBuffer, dimIndex); dims[dimIndex] = dim; } return new IncrementalIndexRow(timeStamp, dims, dimensionDescsList, rowIndex); 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 index 8be93272db03..44be288bb8f6 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeysComparator.java @@ -99,8 +99,8 @@ public int compareKeys(IncrementalIndexRow lhs, IncrementalIndexRow rhs) @Override public int compareSerializedKeys(ByteBuffer lhs, ByteBuffer rhs) { - return compare(new OakIncrementalIndexRow(lhs, null, null), - new OakIncrementalIndexRow(rhs, null, null)); + return compare(new OakBufferIncrementalIndexRow(lhs), + new OakBufferIncrementalIndexRow(rhs)); } @@ -119,7 +119,7 @@ private int lastCompare(int lsIndex, int rsIndex) @Override public int compareSerializedKeyAndKey(ByteBuffer lhs, IncrementalIndexRow rhs) { - return compare(new OakIncrementalIndexRow(lhs, null, null), rhs); + return compare(new OakBufferIncrementalIndexRow(lhs), rhs); } private static boolean allNull(IncrementalIndexRow row, int startPosition) @@ -131,4 +131,47 @@ private static boolean allNull(IncrementalIndexRow row, int startPosition) } return true; } + + private static final class OakBufferIncrementalIndexRow extends IncrementalIndexRow + { + private final ByteBuffer byteBuffer; + + private OakBufferIncrementalIndexRow(ByteBuffer byteBuffer) + { + super(-1, null); + this.byteBuffer = byteBuffer; + } + + @Override + public Object getDim(int dimIndex) + { + return OakUtils.getDimValue(byteBuffer, dimIndex); + } + + //Faster to check this way if dim is null instead of deserializing + @Override + boolean isDimNull(int index) + { + return byteBuffer.getInt(OakUtils.getDimIndexInBuffer(index)) == OakUtils.NO_DIM; + } + + @Override + public long getTimestamp() + { + return OakUtils.getTimestamp(byteBuffer); + } + + @Override + public int getDimsLength() + { + return OakUtils.getDimsLength(byteBuffer); + } + + @Override + public int getRowIndex() + { + return OakUtils.getRowIndex(byteBuffer); + } + } + } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java index 919794f6ad78..00b0d6b2f4ba 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.incremental; +import com.oath.oak.OakRBuffer; import org.apache.druid.segment.column.ValueType; import java.nio.ByteBuffer; @@ -40,20 +41,29 @@ private OakUtils() { } + static long getTimestamp(OakRBuffer buff) + { + return buff.getLong(TIME_STAMP_INDEX); + } + static long getTimestamp(ByteBuffer buff) { return buff.getLong(buff.position() + TIME_STAMP_INDEX); } + static int getRowIndex(OakRBuffer buff) + { + return buff.getInt(ROW_INDEX_INDEX); + } + static int getRowIndex(ByteBuffer buff) { return buff.getInt(buff.position() + ROW_INDEX_INDEX); } - static Object getDimValue(ByteBuffer buff, int dimIndex) + static int getDimsLength(OakRBuffer buff) { - int dimsLength = getDimsLength(buff); - return getDimValue(buff, dimIndex, dimsLength); + return buff.getInt(DIMS_LENGTH_INDEX); } static int getDimsLength(ByteBuffer buff) @@ -61,21 +71,42 @@ static int getDimsLength(ByteBuffer buff) return buff.getInt(buff.position() + DIMS_LENGTH_INDEX); } - static int getDimIndexInBuffer(ByteBuffer buff, int dimsLength, int dimIndex) + static int getDimIndexInBuffer(int dimIndex) { - if (dimIndex >= dimsLength) { - return NO_DIM; - } - return buff.position() + DIMS_INDEX + dimIndex * ALLOC_PER_DIM; + return DIMS_INDEX + dimIndex * ALLOC_PER_DIM; } - static Object getDimValue(ByteBuffer buff, int dimIndex, int dimsLength) + static Object getDimValue(OakRBuffer buff, int dimIndex) { Object dimObject = null; - if (dimIndex >= dimsLength) { + int dimIndexInBuffer = getDimIndexInBuffer(dimIndex); + int dimType = buff.getInt(dimIndexInBuffer); + if (dimType == NO_DIM) { return null; + } else if (dimType == ValueType.DOUBLE.ordinal()) { + dimObject = buff.getDouble(dimIndexInBuffer + DATA_OFFSET); + } else if (dimType == ValueType.FLOAT.ordinal()) { + dimObject = buff.getFloat(dimIndexInBuffer + DATA_OFFSET); + } else if (dimType == ValueType.LONG.ordinal()) { + dimObject = buff.getLong(dimIndexInBuffer + DATA_OFFSET); + } else if (dimType == ValueType.STRING.ordinal()) { + int arrayIndexOffset = buff.getInt(dimIndexInBuffer + ARRAY_INDEX_OFFSET); + int arrayIndex = arrayIndexOffset; + int arraySize = buff.getInt(dimIndexInBuffer + 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; } - int dimIndexInBuffer = getDimIndexInBuffer(buff, dimsLength, dimIndex); + return dimObject; + } + + static Object getDimValue(ByteBuffer buff, int dimIndex) + { + Object dimObject = null; + int dimIndexInBuffer = buff.position() + getDimIndexInBuffer(dimIndex); int dimType = buff.getInt(dimIndexInBuffer); if (dimType == NO_DIM) { return null; From 8cce3988688910ab0ddf6285e5cec5c92031f209 Mon Sep 17 00:00:00 2001 From: Eran Meir Date: Thu, 18 Apr 2019 11:30:40 +0300 Subject: [PATCH 09/15] Configuration support for using Oak to use Oak, add the following in middieManager's runtime.properties: druid.indexer.useOak=true --- .../indexing/kafka/KafkaIndexTaskTest.java | 4 +- .../kinesis/KinesisIndexTaskTest.java | 4 +- .../druid/indexing/common/TaskToolbox.java | 61 +++++++++++-------- .../indexing/common/TaskToolboxFactory.java | 58 ++++++++++-------- .../common/index/YeOldePlumberSchool.java | 4 +- .../AppenderatorDriverRealtimeIndexTask.java | 3 +- .../druid/indexing/common/task/IndexTask.java | 3 +- .../common/task/RealtimeIndexTask.java | 3 +- .../batch/parallel/ParallelIndexSubTask.java | 3 +- .../SeekableStreamIndexTask.java | 3 +- .../indexing/common/TaskToolboxTest.java | 10 ++- ...penderatorDriverRealtimeIndexTaskTest.java | 4 +- .../common/task/CompactionTaskRunTest.java | 4 +- .../common/task/CompactionTaskTest.java | 4 +- .../indexing/common/task/IndexTaskTest.java | 4 +- .../common/task/RealtimeIndexTaskTest.java | 4 +- ...stractParallelIndexSupervisorTaskTest.java | 4 +- .../SingleTaskBackgroundRunnerTest.java | 4 +- .../indexing/overlord/TaskLifecycleTest.java | 4 +- .../worker/WorkerTaskManagerTest.java | 4 +- .../worker/WorkerTaskMonitorTest.java | 4 +- .../appenderator/AppenderatorImpl.java | 13 ++-- .../realtime/appenderator/Appenderators.java | 12 ++-- .../DefaultOfflineAppenderatorFactory.java | 8 ++- .../DefaultRealtimeAppenderatorFactory.java | 9 ++- .../realtime/plumber/FlushingPlumber.java | 4 +- .../plumber/FlushingPlumberSchool.java | 28 +++++---- .../realtime/plumber/RealtimePlumber.java | 44 ++++++------- .../plumber/RealtimePlumberSchool.java | 10 ++- .../druid/segment/realtime/plumber/Sink.java | 53 ++++++++-------- .../segment/realtime/FireDepartmentTest.java | 4 +- .../segment/realtime/RealtimeManagerTest.java | 8 +-- .../appenderator/AppenderatorTester.java | 3 +- ...DefaultOfflineAppenderatorFactoryTest.java | 1 + .../plumber/RealtimePlumberSchoolTest.java | 15 ++--- .../segment/realtime/plumber/SinkTest.java | 8 +-- .../java/org/apache/druid/cli/CliPeon.java | 2 + 37 files changed, 236 insertions(+), 182 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 6b9a20365b55..ec298143d237 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2734,8 +2734,8 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new TaskReportFileWriter(reportsFile) - ); + new TaskReportFileWriter(reportsFile), + false); } private void destroyToolboxFactory() diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java index a04556d6abf0..4a2565fa352a 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java @@ -2870,8 +2870,8 @@ public void close() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0), - new TaskReportFileWriter(reportsFile) - ); + new TaskReportFileWriter(reportsFile), + false); } private void destroyToolboxFactory() diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index d4f788aab764..390c4af793ec 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -98,35 +98,36 @@ public class TaskToolbox private final DruidNode druidNode; private final LookupNodeService lookupNodeService; private final DataNodeService dataNodeService; + private final boolean useOak; public TaskToolbox( - TaskConfig config, - TaskActionClient taskActionClient, - ServiceEmitter emitter, - DataSegmentPusher segmentPusher, - DataSegmentKiller dataSegmentKiller, - DataSegmentMover dataSegmentMover, - DataSegmentArchiver dataSegmentArchiver, - DataSegmentAnnouncer segmentAnnouncer, - DataSegmentServerAnnouncer serverAnnouncer, - SegmentHandoffNotifierFactory handoffNotifierFactory, - Provider queryRunnerFactoryConglomerateProvider, - ExecutorService queryExecutorService, - MonitorScheduler monitorScheduler, - SegmentLoader segmentLoader, - ObjectMapper objectMapper, - File taskWorkDir, - IndexIO indexIO, - Cache cache, - CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats, - IndexMergerV9 indexMergerV9, - DruidNodeAnnouncer druidNodeAnnouncer, - DruidNode druidNode, - LookupNodeService lookupNodeService, - DataNodeService dataNodeService, - TaskReportFileWriter taskReportFileWriter - ) + TaskConfig config, + TaskActionClient taskActionClient, + ServiceEmitter emitter, + DataSegmentPusher segmentPusher, + DataSegmentKiller dataSegmentKiller, + DataSegmentMover dataSegmentMover, + DataSegmentArchiver dataSegmentArchiver, + DataSegmentAnnouncer segmentAnnouncer, + DataSegmentServerAnnouncer serverAnnouncer, + SegmentHandoffNotifierFactory handoffNotifierFactory, + Provider queryRunnerFactoryConglomerateProvider, + ExecutorService queryExecutorService, + MonitorScheduler monitorScheduler, + SegmentLoader segmentLoader, + ObjectMapper objectMapper, + File taskWorkDir, + IndexIO indexIO, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats, + IndexMergerV9 indexMergerV9, + DruidNodeAnnouncer druidNodeAnnouncer, + DruidNode druidNode, + LookupNodeService lookupNodeService, + DataNodeService dataNodeService, + TaskReportFileWriter taskReportFileWriter, + boolean useOak) { this.config = config; this.taskActionClient = taskActionClient; @@ -155,6 +156,7 @@ public TaskToolbox( this.dataNodeService = dataNodeService; this.taskReportFileWriter = taskReportFileWriter; this.taskReportFileWriter.setObjectMapper(this.objectMapper); + this.useOak = useOak; } public TaskConfig getConfig() @@ -321,4 +323,9 @@ public TaskReportFileWriter getTaskReportFileWriter() { return taskReportFileWriter; } + + public boolean isUseOak() + { + return useOak; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 316136df7443..452e78863706 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -23,6 +23,7 @@ import com.google.common.base.Preconditions; import com.google.inject.Inject; import com.google.inject.Provider; +import com.google.inject.name.Named; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; @@ -81,35 +82,36 @@ public class TaskToolboxFactory private final LookupNodeService lookupNodeService; private final DataNodeService dataNodeService; private final TaskReportFileWriter taskReportFileWriter; + private final boolean useOak; @Inject public TaskToolboxFactory( - TaskConfig config, - TaskActionClientFactory taskActionClientFactory, - ServiceEmitter emitter, - DataSegmentPusher segmentPusher, - DataSegmentKiller dataSegmentKiller, - DataSegmentMover dataSegmentMover, - DataSegmentArchiver dataSegmentArchiver, - DataSegmentAnnouncer segmentAnnouncer, - DataSegmentServerAnnouncer serverAnnouncer, - SegmentHandoffNotifierFactory handoffNotifierFactory, - Provider queryRunnerFactoryConglomerateProvider, - @Processing ExecutorService queryExecutorService, - MonitorScheduler monitorScheduler, - SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper, - IndexIO indexIO, - Cache cache, - CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats, - IndexMergerV9 indexMergerV9, - DruidNodeAnnouncer druidNodeAnnouncer, - @RemoteChatHandler DruidNode druidNode, - LookupNodeService lookupNodeService, - DataNodeService dataNodeService, - TaskReportFileWriter taskReportFileWriter - ) + TaskConfig config, + TaskActionClientFactory taskActionClientFactory, + ServiceEmitter emitter, + DataSegmentPusher segmentPusher, + DataSegmentKiller dataSegmentKiller, + DataSegmentMover dataSegmentMover, + DataSegmentArchiver dataSegmentArchiver, + DataSegmentAnnouncer segmentAnnouncer, + DataSegmentServerAnnouncer serverAnnouncer, + SegmentHandoffNotifierFactory handoffNotifierFactory, + Provider queryRunnerFactoryConglomerateProvider, + @Processing ExecutorService queryExecutorService, + MonitorScheduler monitorScheduler, + SegmentLoaderFactory segmentLoaderFactory, + ObjectMapper objectMapper, + IndexIO indexIO, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats, + IndexMergerV9 indexMergerV9, + DruidNodeAnnouncer druidNodeAnnouncer, + @RemoteChatHandler DruidNode druidNode, + LookupNodeService lookupNodeService, + DataNodeService dataNodeService, + TaskReportFileWriter taskReportFileWriter, + @Named("useOak") boolean useOak) { this.config = config; this.taskActionClientFactory = taskActionClientFactory; @@ -136,6 +138,7 @@ public TaskToolboxFactory( this.lookupNodeService = lookupNodeService; this.dataNodeService = dataNodeService; this.taskReportFileWriter = taskReportFileWriter; + this.useOak = useOak; } public TaskToolbox build(Task task) @@ -167,7 +170,8 @@ public TaskToolbox build(Task task) druidNode, lookupNodeService, dataNodeService, - taskReportFileWriter + taskReportFileWriter, + useOak ); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java index 6e0e1d861d16..0ff747d8ae26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/index/YeOldePlumberSchool.java @@ -108,8 +108,8 @@ public Plumber findPlumber( config.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), config.isReportParseExceptions(), - config.getDedupColumn() - ); + config.getDedupColumn(), + false); // Temporary directory to hold spilled segments. final File persistDir = new File(tmpSegmentDir, theSink.getSegment().getId().toString()); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 19a256883edd..0ca9f2ed253e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -703,7 +703,8 @@ private static Appenderator newAppenderator( toolbox.getQueryExecutorService(), toolbox.getCache(), toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats() + toolbox.getCachePopulatorStats(), + toolbox.isUseOak() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 5161f9bcba93..cd96525095e4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -1117,7 +1117,8 @@ private static Appenderator newAppenderator( toolbox.getSegmentPusher(), toolbox.getObjectMapper(), toolbox.getIndexIO(), - toolbox.getIndexMergerV9() + toolbox.getIndexMergerV9(), + toolbox.isUseOak() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 2786805b9f4c..40bd94499038 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -350,7 +350,8 @@ public String getVersion(final Interval interval) toolbox.getCache(), toolbox.getCacheConfig(), toolbox.getCachePopulatorStats(), - toolbox.getObjectMapper() + toolbox.getObjectMapper(), + toolbox.isUseOak() ); this.plumber = plumberSchool.findPlumber(dataSchema, tuningConfig, metrics); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java index 435de05892ff..3615776e8e59 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSubTask.java @@ -391,7 +391,8 @@ private static Appenderator newAppenderator( toolbox.getSegmentPusher(), toolbox.getObjectMapper(), toolbox.getIndexIO(), - toolbox.getIndexMergerV9() + toolbox.getIndexMergerV9(), + toolbox.isUseOak() ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java index 833cc172787a..0093778c0893 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTask.java @@ -208,7 +208,8 @@ public Appenderator newAppenderator(FireDepartmentMetrics metrics, TaskToolbox t toolbox.getQueryExecutorService(), toolbox.getCache(), toolbox.getCacheConfig(), - toolbox.getCachePopulatorStats() + toolbox.getCachePopulatorStats(), + toolbox.isUseOak() ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java index 0966d1b84c98..91e95a86d987 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/TaskToolboxTest.java @@ -119,8 +119,8 @@ public void setUp() throws IOException null, null, null, - new NoopTestTaskFileWriter() - ); + new NoopTestTaskFileWriter(), + false); } @Test @@ -207,4 +207,10 @@ public void testGetCacheConfig() { Assert.assertEquals(mockCacheConfig, taskToolbox.build(task).getCacheConfig()); } + + @Test + public void testIsUseOak() + { + Assert.assertEquals(false, taskToolbox.build(task).isUseOak()); + } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index 413e1228d706..f975a5ac58b0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -1621,8 +1621,8 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new TaskReportFileWriter(reportsFile) - ); + new TaskReportFileWriter(reportsFile), + false); } @Nullable diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index ea109c8b6cdc..93196d7e08ad 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -436,8 +436,8 @@ public File getPath() null, null, null, - new NoopTestTaskFileWriter() - ); + new NoopTestTaskFileWriter(), + false); if (task.isReady(box.getTaskActionClient())) { TaskStatus status = task.run(box); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 74b8b4b0e22d..b7d4920026cc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1367,8 +1367,8 @@ private static class TestTaskToolbox extends TaskToolbox null, null, null, - new NoopTestTaskFileWriter() - ); + new NoopTestTaskFileWriter(), + false); this.segmentFileMap = segmentFileMap; } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java index 97279f94b643..7d3807bf4b20 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IndexTaskTest.java @@ -1578,8 +1578,8 @@ public void killAll() null, null, null, - new TaskReportFileWriter(reportsFile) - ); + new TaskReportFileWriter(reportsFile), + false); indexTask.isReady(box.getTaskActionClient()); TaskStatus status = indexTask.run(box); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index e2a24f582377..5d164be5a024 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -999,8 +999,8 @@ public List getLocations() EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskFileWriter() - ); + new NoopTestTaskFileWriter(), + false); return toolboxFactory.build(task); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 8792956bc95a..d93b5e19c8a4 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -262,8 +262,8 @@ public void killAll() null, null, null, - new NoopTestTaskFileWriter() - ); + new NoopTestTaskFileWriter(), + false); } static class TestParallelIndexSupervisorTask extends ParallelIndexSupervisorTask diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java index 49315d3973d0..feb3c7ae9ded 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/SingleTaskBackgroundRunnerTest.java @@ -104,8 +104,8 @@ public void setup() throws IOException node, null, null, - new TaskReportFileWriter(new File("fake")) - ); + new TaskReportFileWriter(new File("fake")), + false); runner = new SingleTaskBackgroundRunner( toolboxFactory, taskConfig, diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 6f2b99218a26..6bc05eb06861 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -620,8 +620,8 @@ public void unannounceSegments(Iterable segments) EasyMock.createNiceMock(DruidNode.class), new LookupNodeService("tier"), new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0), - new NoopTestTaskFileWriter() - ); + new NoopTestTaskFileWriter(), + false); } private TaskRunner setUpThreadPoolTaskRunner(TaskToolboxFactory tb) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java index b86b654847a4..6d0f5987c509 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskManagerTest.java @@ -130,8 +130,8 @@ public List getLocations() null, null, null, - new NoopTestTaskFileWriter() - ), + new NoopTestTaskFileWriter(), + false), taskConfig, location ), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java index 4afdd3cf9d5e..dcf8afff8ead 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -178,8 +178,8 @@ private WorkerTaskMonitor createTaskMonitor() null, null, null, - new NoopTestTaskFileWriter() - ), + new NoopTestTaskFileWriter(), + false), taskConfig, new NoopServiceEmitter(), DUMMY_NODE, diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index 647340bbb502..e64279f2ae4f 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -156,6 +156,7 @@ public class AppenderatorImpl implements Appenderator private volatile FileChannel basePersistDirLockChannel = null; private volatile Throwable persistError; + private boolean useOak; AppenderatorImpl( DataSchema schema, @@ -171,7 +172,8 @@ public class AppenderatorImpl implements Appenderator IndexMerger indexMerger, Cache cache, CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats + CachePopulatorStats cachePopulatorStats, + boolean useOak ) { this.schema = Preconditions.checkNotNull(schema, "schema"); @@ -195,6 +197,7 @@ public class AppenderatorImpl implements Appenderator cachePopulatorStats ); maxBytesTuningConfig = TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()); + this.useOak = useOak; log.info("Created Appenderator for dataSource[%s].", schema.getDataSource()); } @@ -391,8 +394,8 @@ private Sink getOrCreateSink(final SegmentIdWithShardSpec identifier) tuningConfig.getMaxRowsInMemory(), maxBytesTuningConfig, tuningConfig.isReportParseExceptions(), - null - ); + null, + useOak); try { segmentAnnouncer.announceSegment(retVal.getSegment()); @@ -1058,8 +1061,8 @@ private Object bootstrapSinksFromDisk() maxBytesTuningConfig, tuningConfig.isReportParseExceptions(), null, - hydrants - ); + hydrants, + false); rowsSoFar += currSink.getNumRows(); sinks.put(identifier, currSink); sinkTimeline.add( diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java index 44d91cb54416..ff9a4e1b9b8d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/Appenderators.java @@ -54,7 +54,8 @@ public static Appenderator createRealtime( ExecutorService queryExecutorService, Cache cache, CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats + CachePopulatorStats cachePopulatorStats, + boolean useOak ) { return new AppenderatorImpl( @@ -71,7 +72,8 @@ public static Appenderator createRealtime( indexMerger, cache, cacheConfig, - cachePopulatorStats + cachePopulatorStats, + useOak ); } @@ -82,7 +84,8 @@ public static Appenderator createOffline( DataSegmentPusher dataSegmentPusher, ObjectMapper objectMapper, IndexIO indexIO, - IndexMerger indexMerger + IndexMerger indexMerger, + boolean useOak ) { return new AppenderatorImpl( @@ -124,7 +127,8 @@ public void unannounceSegments(Iterable segments) indexMerger, null, null, - null + null, + useOak ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java index be85e2c9d271..688042a38299 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactory.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.name.Named; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.indexing.DataSchema; @@ -36,24 +37,27 @@ public class DefaultOfflineAppenderatorFactory implements AppenderatorFactory private final ObjectMapper objectMapper; private final IndexIO indexIO; private final IndexMerger indexMerger; + private final boolean useOak; @JsonCreator public DefaultOfflineAppenderatorFactory( @JacksonInject DataSegmentPusher dataSegmentPusher, @JacksonInject ObjectMapper objectMapper, @JacksonInject IndexIO indexIO, - @JacksonInject IndexMerger indexMerger + @JacksonInject IndexMerger indexMerger, + @JacksonInject @Named("useOak") boolean useOak ) { this.dataSegmentPusher = dataSegmentPusher; this.objectMapper = objectMapper; this.indexIO = indexIO; this.indexMerger = indexMerger; + this.useOak = useOak; } @Override public Appenderator build(DataSchema schema, RealtimeTuningConfig config, FireDepartmentMetrics metrics) { - return Appenderators.createOffline(schema, config, metrics, dataSegmentPusher, objectMapper, indexIO, indexMerger); + return Appenderators.createOffline(schema, config, metrics, dataSegmentPusher, objectMapper, indexIO, indexMerger, useOak); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java index ec1abe3f9997..174d8930cf86 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/DefaultRealtimeAppenderatorFactory.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.name.Named; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; @@ -52,6 +53,7 @@ public class DefaultRealtimeAppenderatorFactory implements AppenderatorFactory private final Cache cache; private final CacheConfig cacheConfig; private final CachePopulatorStats cachePopulatorStats; + private final boolean useOak; public DefaultRealtimeAppenderatorFactory( @JacksonInject ServiceEmitter emitter, @@ -64,7 +66,8 @@ public DefaultRealtimeAppenderatorFactory( @JacksonInject IndexMerger indexMerger, @JacksonInject Cache cache, @JacksonInject CacheConfig cacheConfig, - @JacksonInject CachePopulatorStats cachePopulatorStats + @JacksonInject CachePopulatorStats cachePopulatorStats, + @JacksonInject @Named("useOak") boolean useOak ) { this.emitter = emitter; @@ -78,6 +81,7 @@ public DefaultRealtimeAppenderatorFactory( this.cache = cache; this.cacheConfig = cacheConfig; this.cachePopulatorStats = cachePopulatorStats; + this.useOak = useOak; } @Override @@ -107,7 +111,8 @@ public Appenderator build( queryExecutorService, cache, cacheConfig, - cachePopulatorStats + cachePopulatorStats, + useOak ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumber.java index fca84bee6388..94ff8abd49a9 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumber.java @@ -95,8 +95,8 @@ public FlushingPlumber( cache, cacheConfig, cachePopulatorStats, - objectMapper - ); + objectMapper, + false); this.flushDuration = flushDuration; this.schema = schema; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 1c7d9827d844..01a5c8064ebc 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.inject.name.Named; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; @@ -63,18 +64,18 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool @JsonCreator public FlushingPlumberSchool( - @JsonProperty("flushDuration") Duration flushDuration, - @JacksonInject ServiceEmitter emitter, - @JacksonInject QueryRunnerFactoryConglomerate conglomerate, - @JacksonInject DataSegmentAnnouncer segmentAnnouncer, - @JacksonInject @Processing ExecutorService queryExecutorService, - @JacksonInject IndexMergerV9 indexMergerV9, - @JacksonInject IndexIO indexIO, - @JacksonInject Cache cache, - @JacksonInject CacheConfig cacheConfig, - @JacksonInject CachePopulatorStats cachePopulatorStats, - @JacksonInject ObjectMapper objectMapper - ) + @JsonProperty("flushDuration") Duration flushDuration, + @JacksonInject ServiceEmitter emitter, + @JacksonInject QueryRunnerFactoryConglomerate conglomerate, + @JacksonInject DataSegmentAnnouncer segmentAnnouncer, + @JacksonInject @Processing ExecutorService queryExecutorService, + @JacksonInject IndexMergerV9 indexMergerV9, + @JacksonInject IndexIO indexIO, + @JacksonInject Cache cache, + @JacksonInject CacheConfig cacheConfig, + @JacksonInject CachePopulatorStats cachePopulatorStats, + @JacksonInject ObjectMapper objectMapper, + @JacksonInject @Named("useOak") boolean useOak) { super( emitter, @@ -89,7 +90,8 @@ public FlushingPlumberSchool( cache, cacheConfig, cachePopulatorStats, - objectMapper + objectMapper, + useOak ); this.flushDuration = flushDuration == null ? defaultFlushDuration : flushDuration; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index 8e0ad2b999f5..3a6e0379da29 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -130,25 +130,26 @@ public class RealtimePlumber implements Plumber private static final String COMMIT_METADATA_KEY = "%commitMetadata%"; private static final String COMMIT_METADATA_TIMESTAMP_KEY = "%commitMetadataTimestamp%"; + private final boolean useOak; public RealtimePlumber( - DataSchema schema, - RealtimeTuningConfig config, - FireDepartmentMetrics metrics, - ServiceEmitter emitter, - QueryRunnerFactoryConglomerate conglomerate, - DataSegmentAnnouncer segmentAnnouncer, - ExecutorService queryExecutorService, - DataSegmentPusher dataSegmentPusher, - SegmentPublisher segmentPublisher, - SegmentHandoffNotifier handoffNotifier, - IndexMerger indexMerger, - IndexIO indexIO, - Cache cache, - CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats, - ObjectMapper objectMapper - ) + DataSchema schema, + RealtimeTuningConfig config, + FireDepartmentMetrics metrics, + ServiceEmitter emitter, + QueryRunnerFactoryConglomerate conglomerate, + DataSegmentAnnouncer segmentAnnouncer, + ExecutorService queryExecutorService, + DataSegmentPusher dataSegmentPusher, + SegmentPublisher segmentPublisher, + SegmentHandoffNotifier handoffNotifier, + IndexMerger indexMerger, + IndexIO indexIO, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats, + ObjectMapper objectMapper, + boolean useOak) { this.schema = schema; this.config = config; @@ -172,6 +173,7 @@ public RealtimePlumber( cacheConfig, cachePopulatorStats ); + this.useOak = useOak; log.info("Creating plumber using rejectionPolicy[%s]", getRejectionPolicy()); } @@ -260,8 +262,8 @@ private Sink getSink(long timestamp) config.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), config.isReportParseExceptions(), - config.getDedupColumn() - ); + config.getDedupColumn(), + useOak); addSink(retVal); } @@ -723,8 +725,8 @@ public int compare(File o1, File o2) TuningConfigs.getMaxBytesInMemoryOrDefault(config.getMaxBytesInMemory()), config.isReportParseExceptions(), config.getDedupColumn(), - hydrants - ); + hydrants, + useOak); addSink(currSink); } return metadata; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java index 863fa58c2739..fb4e80f124c4 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchool.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import com.google.inject.name.Named; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; @@ -57,6 +58,7 @@ public class RealtimePlumberSchool implements PlumberSchool private final CacheConfig cacheConfig; private final CachePopulatorStats cachePopulatorStats; private final ObjectMapper objectMapper; + private final boolean useOak; @JsonCreator public RealtimePlumberSchool( @@ -72,7 +74,8 @@ public RealtimePlumberSchool( @JacksonInject Cache cache, @JacksonInject CacheConfig cacheConfig, @JacksonInject CachePopulatorStats cachePopulatorStats, - @JacksonInject ObjectMapper objectMapper + @JacksonInject ObjectMapper objectMapper, + @JacksonInject @Named("useOak") boolean useOak ) { this.emitter = emitter; @@ -89,6 +92,7 @@ public RealtimePlumberSchool( this.cacheConfig = cacheConfig; this.cachePopulatorStats = cachePopulatorStats; this.objectMapper = objectMapper; + this.useOak = useOak; } @Override @@ -116,8 +120,8 @@ public Plumber findPlumber( cache, cacheConfig, cachePopulatorStats, - objectMapper - ); + objectMapper, + useOak); } private void verifyState() diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java index 4e0c596cbdca..5887a8b55024 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/Sink.java @@ -72,17 +72,18 @@ public class Sink implements Iterable private volatile boolean writable = true; private final String dedupColumn; private final Set dedupSet = new HashSet<>(); + private final boolean useOak; public Sink( - Interval interval, - DataSchema schema, - ShardSpec shardSpec, - String version, - int maxRowsInMemory, - long maxBytesInMemory, - boolean reportParseExceptions, - String dedupColumn - ) + Interval interval, + DataSchema schema, + ShardSpec shardSpec, + String version, + int maxRowsInMemory, + long maxBytesInMemory, + boolean reportParseExceptions, + String dedupColumn, + boolean useOak) { this.schema = schema; this.shardSpec = shardSpec; @@ -92,21 +93,22 @@ public Sink( this.maxBytesInMemory = maxBytesInMemory; this.reportParseExceptions = reportParseExceptions; this.dedupColumn = dedupColumn; + this.useOak = useOak; makeNewCurrIndex(interval.getStartMillis(), schema); } public Sink( - Interval interval, - DataSchema schema, - ShardSpec shardSpec, - String version, - int maxRowsInMemory, - long maxBytesInMemory, - boolean reportParseExceptions, - String dedupColumn, - List hydrants - ) + Interval interval, + DataSchema schema, + ShardSpec shardSpec, + String version, + int maxRowsInMemory, + long maxBytesInMemory, + boolean reportParseExceptions, + String dedupColumn, + List hydrants, + boolean useOak) { this.schema = schema; this.shardSpec = shardSpec; @@ -116,6 +118,7 @@ public Sink( this.maxBytesInMemory = maxBytesInMemory; this.reportParseExceptions = reportParseExceptions; this.dedupColumn = dedupColumn; + this.useOak = useOak; int maxCount = -1; for (int i = 0; i < hydrants.size(); ++i) { @@ -329,12 +332,12 @@ private FireHydrant makeNewCurrIndex(long minTimestamp, DataSchema schema) .withMetrics(schema.getAggregators()) .withRollup(schema.getGranularitySpec().isRollup()) .build(); - final IncrementalIndex newIndex = new IncrementalIndex.Builder() - .setIndexSchema(indexSchema) - .setReportParseExceptions(reportParseExceptions) - .setMaxRowCount(maxRowsInMemory) - .setMaxBytesInMemory(maxBytesInMemory) - .buildOnheap(); + IncrementalIndex.Builder indexBuilder = new IncrementalIndex.Builder() + .setIndexSchema(indexSchema) + .setReportParseExceptions(reportParseExceptions) + .setMaxRowCount(maxRowsInMemory) + .setMaxBytesInMemory(maxBytesInMemory); + final IncrementalIndex newIndex = useOak ? indexBuilder.buildOak() : indexBuilder.buildOnheap(); final FireHydrant old; synchronized (hydrantLock) { diff --git a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java index 9c373bf97daf..c603d64ed05e 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/FireDepartmentTest.java @@ -117,8 +117,8 @@ public void testSerde() throws Exception MapCache.create(0), NO_CACHE_CONFIG, new CachePopulatorStats(), - TestHelper.makeJsonMapper() - + TestHelper.makeJsonMapper(), + false ), null ), diff --git a/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java index 7df4c14984f3..550814010f10 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/RealtimeManagerTest.java @@ -220,8 +220,8 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), - tuningConfig.getDedupColumn() - )); + tuningConfig.getDedupColumn(), + false)); realtimeManager = new RealtimeManager( Collections.singletonList( @@ -242,8 +242,8 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws ParseExcept tuningConfig.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), - tuningConfig.getDedupColumn() - )); + tuningConfig.getDedupColumn(), + false)); realtimeManager2 = new RealtimeManager( Collections.singletonList( diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java index af706dd53405..c59ffeb94ee6 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -289,7 +289,8 @@ public void unannounceSegments(Iterable segments) queryExecutor, MapCache.create(2048), new CacheConfig(), - new CachePopulatorStats() + new CachePopulatorStats(), + false ); } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index e02ca6d657bc..46b6354a86a7 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -72,6 +72,7 @@ public void configure(Binder binder) binder.bindConstant().annotatedWith(Names.named("serviceName")).to("druid/tool"); binder.bindConstant().annotatedWith(Names.named("servicePort")).to(9999); binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); + binder.bindConstant().annotatedWith(Names.named("useOak")).to(false); binder.bind(DruidProcessingConfig.class).toInstance( new DruidProcessingConfig() { diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index 4052cc678f4a..dcea417e0d2e 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -229,7 +229,8 @@ public void setUp() throws Exception MapCache.create(0), FireDepartmentTest.NO_CACHE_CONFIG, new CachePopulatorStats(), - TestHelper.makeJsonMapper() + TestHelper.makeJsonMapper(), + false ); metrics = new FireDepartmentMetrics(); @@ -275,8 +276,8 @@ private void testPersist(final Object commitMetadata) throws Exception tuningConfig.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), - tuningConfig.getDedupColumn() - ); + tuningConfig.getDedupColumn(), + false); plumber.getSinks().put(0L, sink); Assert.assertNull(plumber.startJob()); @@ -321,8 +322,8 @@ public void testPersistFails() throws Exception tuningConfig.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), - tuningConfig.getDedupColumn() - ); + tuningConfig.getDedupColumn(), + false); plumber.getSinks().put(0L, sink); plumber.startJob(); final InputRow row = EasyMock.createNiceMock(InputRow.class); @@ -377,8 +378,8 @@ private void testPersistHydrantGapsHelper(final Object commitMetadata) throws Ex tuningConfig.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), - tuningConfig.getDedupColumn() - ); + tuningConfig.getDedupColumn(), + false); plumber2.getSinks().put(0L, sink); Assert.assertNull(plumber2.startJob()); final CountDownLatch doneSignal = new CountDownLatch(1); diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java index de55360a61d2..05af3387b199 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/SinkTest.java @@ -91,8 +91,8 @@ public void testSwap() throws Exception tuningConfig.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), - tuningConfig.getDedupColumn() - ); + tuningConfig.getDedupColumn(), + false); sink.add( new InputRow() @@ -245,8 +245,8 @@ public void testDedup() throws Exception tuningConfig.getMaxRowsInMemory(), TuningConfigs.getMaxBytesInMemoryOrDefault(tuningConfig.getMaxBytesInMemory()), tuningConfig.isReportParseExceptions(), - tuningConfig.getDedupColumn() - ); + tuningConfig.getDedupColumn(), + false); int rows = sink.add(new MapBasedInputRow( DateTimes.of("2013-01-01"), diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index ba64d72313fd..878635036c23 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -200,6 +200,8 @@ public void configure(Binder binder) binder.bind(NoopChatHandlerProvider.class).in(LazySingleton.class); + binder.bindConstant().annotatedWith(Names.named("useOak")).to(Boolean.parseBoolean(properties.getProperty("druid.indexer.useOak"))); + binder.bind(TaskToolboxFactory.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class); From 2c23344a20afef8703e73d6ffb45df1c8e6fbf77 Mon Sep 17 00:00:00 2001 From: Eran Meir Date: Sun, 21 Apr 2019 12:01:43 +0300 Subject: [PATCH 10/15] Unnecessary whitespaces redacted --- .../druid/indexing/common/TaskToolbox.java | 54 +++++++++---------- .../indexing/common/TaskToolboxFactory.java | 52 +++++++++--------- .../plumber/FlushingPlumberSchool.java | 24 ++++----- 3 files changed, 65 insertions(+), 65 deletions(-) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java index 390c4af793ec..7e466b0693b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolbox.java @@ -101,33 +101,33 @@ public class TaskToolbox private final boolean useOak; public TaskToolbox( - TaskConfig config, - TaskActionClient taskActionClient, - ServiceEmitter emitter, - DataSegmentPusher segmentPusher, - DataSegmentKiller dataSegmentKiller, - DataSegmentMover dataSegmentMover, - DataSegmentArchiver dataSegmentArchiver, - DataSegmentAnnouncer segmentAnnouncer, - DataSegmentServerAnnouncer serverAnnouncer, - SegmentHandoffNotifierFactory handoffNotifierFactory, - Provider queryRunnerFactoryConglomerateProvider, - ExecutorService queryExecutorService, - MonitorScheduler monitorScheduler, - SegmentLoader segmentLoader, - ObjectMapper objectMapper, - File taskWorkDir, - IndexIO indexIO, - Cache cache, - CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats, - IndexMergerV9 indexMergerV9, - DruidNodeAnnouncer druidNodeAnnouncer, - DruidNode druidNode, - LookupNodeService lookupNodeService, - DataNodeService dataNodeService, - TaskReportFileWriter taskReportFileWriter, - boolean useOak) + TaskConfig config, + TaskActionClient taskActionClient, + ServiceEmitter emitter, + DataSegmentPusher segmentPusher, + DataSegmentKiller dataSegmentKiller, + DataSegmentMover dataSegmentMover, + DataSegmentArchiver dataSegmentArchiver, + DataSegmentAnnouncer segmentAnnouncer, + DataSegmentServerAnnouncer serverAnnouncer, + SegmentHandoffNotifierFactory handoffNotifierFactory, + Provider queryRunnerFactoryConglomerateProvider, + ExecutorService queryExecutorService, + MonitorScheduler monitorScheduler, + SegmentLoader segmentLoader, + ObjectMapper objectMapper, + File taskWorkDir, + IndexIO indexIO, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats, + IndexMergerV9 indexMergerV9, + DruidNodeAnnouncer druidNodeAnnouncer, + DruidNode druidNode, + LookupNodeService lookupNodeService, + DataNodeService dataNodeService, + TaskReportFileWriter taskReportFileWriter, + boolean useOak) { this.config = config; this.taskActionClient = taskActionClient; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java index 452e78863706..241c5132c77b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/TaskToolboxFactory.java @@ -86,32 +86,32 @@ public class TaskToolboxFactory @Inject public TaskToolboxFactory( - TaskConfig config, - TaskActionClientFactory taskActionClientFactory, - ServiceEmitter emitter, - DataSegmentPusher segmentPusher, - DataSegmentKiller dataSegmentKiller, - DataSegmentMover dataSegmentMover, - DataSegmentArchiver dataSegmentArchiver, - DataSegmentAnnouncer segmentAnnouncer, - DataSegmentServerAnnouncer serverAnnouncer, - SegmentHandoffNotifierFactory handoffNotifierFactory, - Provider queryRunnerFactoryConglomerateProvider, - @Processing ExecutorService queryExecutorService, - MonitorScheduler monitorScheduler, - SegmentLoaderFactory segmentLoaderFactory, - ObjectMapper objectMapper, - IndexIO indexIO, - Cache cache, - CacheConfig cacheConfig, - CachePopulatorStats cachePopulatorStats, - IndexMergerV9 indexMergerV9, - DruidNodeAnnouncer druidNodeAnnouncer, - @RemoteChatHandler DruidNode druidNode, - LookupNodeService lookupNodeService, - DataNodeService dataNodeService, - TaskReportFileWriter taskReportFileWriter, - @Named("useOak") boolean useOak) + TaskConfig config, + TaskActionClientFactory taskActionClientFactory, + ServiceEmitter emitter, + DataSegmentPusher segmentPusher, + DataSegmentKiller dataSegmentKiller, + DataSegmentMover dataSegmentMover, + DataSegmentArchiver dataSegmentArchiver, + DataSegmentAnnouncer segmentAnnouncer, + DataSegmentServerAnnouncer serverAnnouncer, + SegmentHandoffNotifierFactory handoffNotifierFactory, + Provider queryRunnerFactoryConglomerateProvider, + @Processing ExecutorService queryExecutorService, + MonitorScheduler monitorScheduler, + SegmentLoaderFactory segmentLoaderFactory, + ObjectMapper objectMapper, + IndexIO indexIO, + Cache cache, + CacheConfig cacheConfig, + CachePopulatorStats cachePopulatorStats, + IndexMergerV9 indexMergerV9, + DruidNodeAnnouncer druidNodeAnnouncer, + @RemoteChatHandler DruidNode druidNode, + LookupNodeService lookupNodeService, + DataNodeService dataNodeService, + TaskReportFileWriter taskReportFileWriter, + @Named("useOak") boolean useOak) { this.config = config; this.taskActionClientFactory = taskActionClientFactory; diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java index 01a5c8064ebc..f59895d3c82e 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/FlushingPlumberSchool.java @@ -64,18 +64,18 @@ public class FlushingPlumberSchool extends RealtimePlumberSchool @JsonCreator public FlushingPlumberSchool( - @JsonProperty("flushDuration") Duration flushDuration, - @JacksonInject ServiceEmitter emitter, - @JacksonInject QueryRunnerFactoryConglomerate conglomerate, - @JacksonInject DataSegmentAnnouncer segmentAnnouncer, - @JacksonInject @Processing ExecutorService queryExecutorService, - @JacksonInject IndexMergerV9 indexMergerV9, - @JacksonInject IndexIO indexIO, - @JacksonInject Cache cache, - @JacksonInject CacheConfig cacheConfig, - @JacksonInject CachePopulatorStats cachePopulatorStats, - @JacksonInject ObjectMapper objectMapper, - @JacksonInject @Named("useOak") boolean useOak) + @JsonProperty("flushDuration") Duration flushDuration, + @JacksonInject ServiceEmitter emitter, + @JacksonInject QueryRunnerFactoryConglomerate conglomerate, + @JacksonInject DataSegmentAnnouncer segmentAnnouncer, + @JacksonInject @Processing ExecutorService queryExecutorService, + @JacksonInject IndexMergerV9 indexMergerV9, + @JacksonInject IndexIO indexIO, + @JacksonInject Cache cache, + @JacksonInject CacheConfig cacheConfig, + @JacksonInject CachePopulatorStats cachePopulatorStats, + @JacksonInject ObjectMapper objectMapper, + @JacksonInject @Named("useOak") boolean useOak) { super( emitter, From 7b2241d1e4ab5a049f28c77552c97e4edeec4262 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Tue, 16 Apr 2019 17:32:19 +0300 Subject: [PATCH 11/15] minor benchmarks fix --- .../IncrementalIndexReadBenchmark.java | 22 +++++++++++++++++-- .../IndexIngestionMultithreadedBenchmark.java | 18 +++++++++++++++ 2 files changed, 38 insertions(+), 2 deletions(-) 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 c501dd494610..a48baa7f66ba 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 @@ -60,6 +60,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.IOException; import java.util.ArrayList; @@ -141,9 +145,9 @@ private IncrementalIndex makeIncIndex() .setMaxRowCount(rowsPerSegment); switch (indexType) { case "onheap": - builder.buildOnheap(); + return builder.buildOnheap(); case "oak": - builder.buildOak(); + return builder.buildOak(); } return null; } @@ -224,4 +228,18 @@ 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) + .threads(1) + .param("indexType", "oak") + .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 index 6ab4fc80da01..6ce35c7aedaf 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionMultithreadedBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/indexing/IndexIngestionMultithreadedBenchmark.java @@ -43,6 +43,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.util.ArrayList; import java.util.concurrent.TimeUnit; @@ -155,4 +159,18 @@ public void addRows(Blackhole blackhole, ThreadState threadState) throws Excepti blackhole.consume(rv); } } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(IndexIngestionMultithreadedBenchmark.class.getSimpleName()) + .forks(1) + .threads(4) + .param("indexType", "oak") + .param("rollup", "false") + .build(); + + new Runner(opt).run(); + } + } From acdfbbe9a69ff2b71ed9126395cdc147ac7cf950 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Sun, 21 Apr 2019 14:58:06 +0300 Subject: [PATCH 12/15] Avoid double copy of int[] when reading dimention row --- .../druid/segment/StringDimensionIndexer.java | 65 ++++++++++--------- .../segment/data/ArrayBasedIndexedInts.java | 7 ++ .../incremental/IncrementalIndexRow.java | 21 ++++++ .../incremental/OakIncrementalIndexRow.java | 37 +++++++++++ 4 files changed, 101 insertions(+), 29 deletions(-) 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 55bbbfb7b8fb..f5ddd634a98f 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -444,44 +444,51 @@ class IndexerDimensionSelector implements DimensionSelector, IdLookup public IndexedInts getRow() { - int[] indices; - if (dimIndex < currEntry.get().getDimsLength()) { - indices = (int[]) currEntry.get().getDim(dimIndex); + IncrementalIndexRow key = currEntry.get(); + + if (key.avoidDoubleCopyStringDim()) { + // incase of oak the key is serialized so its a wast to copy array twice. + indexedInts.setValues(key, dimIndex); } else { - indices = null; - } + int[] indices; + if (dimIndex < key.getDimsLength()) { + indices = (int[]) currEntry.get().getDim(dimIndex); + } else { + indices = null; + } - int[] row = null; - int rowSize = 0; + int[] row = null; + int rowSize = 0; - // 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 (hasMultipleValues) { - row = IntArrays.EMPTY_ARRAY; - rowSize = 0; - } else { - final int nullId = getEncodedValue(null, false); - if (nullId > -1) { - if (nullIdIntArray == null) { - nullIdIntArray = new int[]{nullId}; - } - row = nullIdIntArray; - rowSize = 1; - } else { - // doesn't contain nullId, then empty array is used - // Choose to use ArrayBasedIndexedInts later, instead of special "empty" IndexedInts, for monomorphism + // 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 (hasMultipleValues) { row = IntArrays.EMPTY_ARRAY; rowSize = 0; + } else { + final int nullId = getEncodedValue(null, false); + if (nullId > -1) { + if (nullIdIntArray == null) { + nullIdIntArray = new int[]{nullId}; + } + row = nullIdIntArray; + rowSize = 1; + } else { + // doesn't contain nullId, then empty array is used + // Choose to use ArrayBasedIndexedInts later, instead of special "empty" IndexedInts, for monomorphism + row = IntArrays.EMPTY_ARRAY; + rowSize = 0; + } } } - } - if (row == null && indices != null && indices.length > 0) { - row = indices; - rowSize = indices.length; - } + if (row == null && indices != null && indices.length > 0) { + row = indices; + rowSize = indices.length; + } - indexedInts.setValues(row, rowSize); + indexedInts.setValues(row, rowSize); + } return indexedInts; } 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..90fa8a127117 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,7 @@ 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 +74,12 @@ public void setValues(int[] values, int size) this.size = size; } + public void setValues(IncrementalIndexRow row, int dimIndex) + { + ensureSize(row.stringDimSize(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/IncrementalIndexRow.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexRow.java index cf3256292edc..3729c91dd8ab 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 @@ -23,6 +23,7 @@ import com.google.common.collect.Lists; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.DimensionIndexer; +import sun.reflect.generics.reflectiveObjects.NotImplementedException; import javax.annotation.Nullable; import java.lang.reflect.Array; @@ -215,4 +216,24 @@ boolean isDimNull(int index) return false; } } + + public boolean avoidDoubleCopyStringDim() + { + return false; + } + + public int stringDimSize(int dimIndex) + { + if (dims == null || dimIndex >= dims.length || dims[dimIndex] == null) { + return 0; + } + + return ((int[]) dims[dimIndex]).length; + } + + public int copyStringDim(int dimIndex, int[] expansion) + { + // No reason to get here + throw new NotImplementedException(); + } } 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 index 116b7dc177de..cc0dfa0e0ecc 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -23,6 +23,7 @@ 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 @@ -111,6 +112,42 @@ public long estimateBytesInMemory() return sizeInBytes; } + @Override + public boolean avoidDoubleCopyStringDim() + { + return true; + } + + @Override + public int stringDimSize(int dimIndex) + { + if (getDimsLength() == 0 || getDimsLength() <= dimIndex) { + return 0; + } + int dimIndexInBuffer = OakUtils.getDimIndexInBuffer(dimIndex); + int arraySize = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_LENGTH_OFFSET); + return arraySize; + } + + @Override + public int copyStringDim(int dimIndex, int[] expansion) + { + if (getDimsLength() == 0 || getDimsLength() <= dimIndex) { + return 0; + } + int dimIndexInBuffer = OakUtils.getDimIndexInBuffer(dimIndex); + + int arrayIndex = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_INDEX_OFFSET); + int arraySize = dimensions.getInt(dimIndexInBuffer + OakUtils.ARRAY_LENGTH_OFFSET); + if (expansion.length < arraySize) { + expansion = new int[arraySize]; + } + for (int i = 0; i < arraySize; i++) { + expansion[i] = dimensions.getInt(arrayIndex); + arrayIndex += Integer.BYTES; + } + return arraySize; + } /* ---------------- OakRBuffer utils -------------- */ From e98ddfb5c9d8aea12fd9f5f88467b76ea0362f13 Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Mon, 22 Apr 2019 12:29:08 +0300 Subject: [PATCH 13/15] Use unsafe memcopy for string dimentions --- .../incremental/OakIncrementalIndexRow.java | 8 ++- .../segment/incremental/OakKeySerializer.java | 4 +- .../druid/segment/incremental/OakUtils.java | 54 ++++++++++++++++--- 3 files changed, 50 insertions(+), 16 deletions(-) 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 index cc0dfa0e0ecc..1b18bad4ae8b 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -23,7 +23,6 @@ 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 @@ -142,10 +141,9 @@ public int copyStringDim(int dimIndex, int[] expansion) if (expansion.length < arraySize) { expansion = new int[arraySize]; } - for (int i = 0; i < arraySize; i++) { - expansion[i] = dimensions.getInt(arrayIndex); - arrayIndex += Integer.BYTES; - } + + dimensions.unsafeBufferToIntArrayCopy(arrayIndex, expansion, arraySize); + return arraySize; } 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 index b265f9bc0c6e..105b7492b72f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java @@ -101,9 +101,7 @@ public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBu int[] arr = (int[]) incrementalIndexRow.getDim(i); byteBuffer.putInt(dimsIndex + arrayIndexOffset, dimsArrayOffset); byteBuffer.putInt(dimsIndex + arrayLengthOffset, arr.length); - for (int arrIndex = 0; arrIndex < arr.length; arrIndex++) { - byteBuffer.putInt(dimsArraysIndex + arrIndex * Integer.BYTES, arr[arrIndex]); - } + OakUtils.unsafeArrayToBufferCopy(byteBuffer, dimsArraysIndex, arr, arr.length); dimsArraysIndex += (arr.length * Integer.BYTES); dimsArrayOffset += (arr.length * Integer.BYTES); break; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java index 00b0d6b2f4ba..105824db043d 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java @@ -21,6 +21,10 @@ import com.oath.oak.OakRBuffer; import org.apache.druid.segment.column.ValueType; +import sun.misc.Unsafe; +import sun.nio.ch.DirectBuffer; + +import java.lang.reflect.Constructor; import java.nio.ByteBuffer; public final class OakUtils @@ -37,6 +41,25 @@ public final class OakUtils static final Integer ARRAY_INDEX_OFFSET = VALUE_TYPE_OFFSET + Integer.BYTES; static final Integer ARRAY_LENGTH_OFFSET = ARRAY_INDEX_OFFSET + Integer.BYTES; + private static Unsafe unsafe; + + private static final long INT_ARRAY_OFFSET; + private static final long BYTE_ARRAY_OFFSET; + + // static constructor - access and create a new instance of Unsafe + static { + try { + Constructor unsafeConstructor = Unsafe.class.getDeclaredConstructor(); + unsafeConstructor.setAccessible(true); + unsafe = unsafeConstructor.newInstance(); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + INT_ARRAY_OFFSET = unsafe.arrayBaseOffset(int[].class); + BYTE_ARRAY_OFFSET = unsafe.arrayBaseOffset(byte[].class); + } + private OakUtils() { } @@ -94,10 +117,7 @@ static Object getDimValue(OakRBuffer buff, int dimIndex) int arrayIndex = arrayIndexOffset; int arraySize = buff.getInt(dimIndexInBuffer + ARRAY_LENGTH_OFFSET); int[] array = new int[arraySize]; - for (int i = 0; i < arraySize; i++) { - array[i] = buff.getInt(arrayIndex); - arrayIndex += Integer.BYTES; - } + buff.unsafeBufferToIntArrayCopy(arrayIndex, array, arraySize); dimObject = array; } return dimObject; @@ -121,12 +141,30 @@ static Object getDimValue(ByteBuffer buff, int dimIndex) int arrayIndex = buff.position() + arrayIndexOffset; int arraySize = buff.getInt(dimIndexInBuffer + ARRAY_LENGTH_OFFSET); int[] array = new int[arraySize]; - for (int i = 0; i < arraySize; i++) { - array[i] = buff.getInt(arrayIndex); - arrayIndex += Integer.BYTES; - } + unsafeBufferToArrayCopy(buff, arrayIndex, array, arraySize); dimObject = array; } return dimObject; } + + static void unsafeBufferToArrayCopy(ByteBuffer srcByteBuffer, int position, int[] dstArray, int countInts) + { + if (srcByteBuffer.isDirect()) { + long bbAddress = ((DirectBuffer) srcByteBuffer).address(); + unsafe.copyMemory(null, bbAddress + position, dstArray, INT_ARRAY_OFFSET, countInts * Integer.BYTES); + } else { + unsafe.copyMemory(srcByteBuffer.array(), BYTE_ARRAY_OFFSET + position, dstArray, INT_ARRAY_OFFSET, countInts * Integer.BYTES); + } + } + + static void unsafeArrayToBufferCopy(ByteBuffer dstByteBuffer, int position, int[] srcArray, int countInts) + { + if (dstByteBuffer.isDirect()) { + long bbAddress = ((DirectBuffer) dstByteBuffer).address(); + unsafe.copyMemory(srcArray, INT_ARRAY_OFFSET, null, bbAddress + position, countInts * Integer.BYTES); + } else { + unsafe.copyMemory(srcArray, INT_ARRAY_OFFSET, dstByteBuffer.array(), BYTE_ARRAY_OFFSET + position, countInts * Integer.BYTES); + } + } + } From 55ace866cd596f1e2d83dff76703fabfd1c8dcdb Mon Sep 17 00:00:00 2001 From: Yonatan Gottesman Date: Sun, 28 Apr 2019 10:38:22 +0300 Subject: [PATCH 14/15] Use unsafe memcopy when writing/reading string dimentions --- .../incremental/OakIncrementalIndexRow.java | 4 +- .../segment/incremental/OakKeySerializer.java | 3 +- .../druid/segment/incremental/OakUtils.java | 49 ++----------------- 3 files changed, 6 insertions(+), 50 deletions(-) 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 index 1b18bad4ae8b..df7cf48d16b1 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakIncrementalIndexRow.java @@ -141,9 +141,7 @@ public int copyStringDim(int dimIndex, int[] expansion) if (expansion.length < arraySize) { expansion = new int[arraySize]; } - - dimensions.unsafeBufferToIntArrayCopy(arrayIndex, expansion, arraySize); - + dimensions.unsafeCopyBufferToIntArray(arrayIndex, expansion, arraySize); return arraySize; } 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 index 105b7492b72f..25a340c6d6c5 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakKeySerializer.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.incremental; import com.oath.oak.OakSerializer; +import com.oath.oak.UnsafeUtils; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ValueType; import java.nio.ByteBuffer; @@ -101,7 +102,7 @@ public void serialize(IncrementalIndexRow incrementalIndexRow, ByteBuffer byteBu int[] arr = (int[]) incrementalIndexRow.getDim(i); byteBuffer.putInt(dimsIndex + arrayIndexOffset, dimsArrayOffset); byteBuffer.putInt(dimsIndex + arrayLengthOffset, arr.length); - OakUtils.unsafeArrayToBufferCopy(byteBuffer, dimsArraysIndex, arr, arr.length); + UnsafeUtils.unsafeCopyIntArrayToBuffer(arr, byteBuffer, dimsArraysIndex, arr.length); dimsArraysIndex += (arr.length * Integer.BYTES); dimsArrayOffset += (arr.length * Integer.BYTES); break; diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java index 105824db043d..e8c12bca3c9f 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/OakUtils.java @@ -20,11 +20,8 @@ package org.apache.druid.segment.incremental; import com.oath.oak.OakRBuffer; +import com.oath.oak.UnsafeUtils; import org.apache.druid.segment.column.ValueType; -import sun.misc.Unsafe; -import sun.nio.ch.DirectBuffer; - -import java.lang.reflect.Constructor; import java.nio.ByteBuffer; public final class OakUtils @@ -41,25 +38,6 @@ public final class OakUtils static final Integer ARRAY_INDEX_OFFSET = VALUE_TYPE_OFFSET + Integer.BYTES; static final Integer ARRAY_LENGTH_OFFSET = ARRAY_INDEX_OFFSET + Integer.BYTES; - private static Unsafe unsafe; - - private static final long INT_ARRAY_OFFSET; - private static final long BYTE_ARRAY_OFFSET; - - // static constructor - access and create a new instance of Unsafe - static { - try { - Constructor unsafeConstructor = Unsafe.class.getDeclaredConstructor(); - unsafeConstructor.setAccessible(true); - unsafe = unsafeConstructor.newInstance(); - } - catch (Exception ex) { - throw new RuntimeException(ex); - } - INT_ARRAY_OFFSET = unsafe.arrayBaseOffset(int[].class); - BYTE_ARRAY_OFFSET = unsafe.arrayBaseOffset(byte[].class); - } - private OakUtils() { } @@ -117,7 +95,7 @@ static Object getDimValue(OakRBuffer buff, int dimIndex) int arrayIndex = arrayIndexOffset; int arraySize = buff.getInt(dimIndexInBuffer + ARRAY_LENGTH_OFFSET); int[] array = new int[arraySize]; - buff.unsafeBufferToIntArrayCopy(arrayIndex, array, arraySize); + buff.unsafeCopyBufferToIntArray(arrayIndex, array, arraySize); dimObject = array; } return dimObject; @@ -141,30 +119,9 @@ static Object getDimValue(ByteBuffer buff, int dimIndex) int arrayIndex = buff.position() + arrayIndexOffset; int arraySize = buff.getInt(dimIndexInBuffer + ARRAY_LENGTH_OFFSET); int[] array = new int[arraySize]; - unsafeBufferToArrayCopy(buff, arrayIndex, array, arraySize); + UnsafeUtils.unsafeCopyBufferToIntArray(buff, arrayIndex, array, arraySize); dimObject = array; } return dimObject; } - - static void unsafeBufferToArrayCopy(ByteBuffer srcByteBuffer, int position, int[] dstArray, int countInts) - { - if (srcByteBuffer.isDirect()) { - long bbAddress = ((DirectBuffer) srcByteBuffer).address(); - unsafe.copyMemory(null, bbAddress + position, dstArray, INT_ARRAY_OFFSET, countInts * Integer.BYTES); - } else { - unsafe.copyMemory(srcByteBuffer.array(), BYTE_ARRAY_OFFSET + position, dstArray, INT_ARRAY_OFFSET, countInts * Integer.BYTES); - } - } - - static void unsafeArrayToBufferCopy(ByteBuffer dstByteBuffer, int position, int[] srcArray, int countInts) - { - if (dstByteBuffer.isDirect()) { - long bbAddress = ((DirectBuffer) dstByteBuffer).address(); - unsafe.copyMemory(srcArray, INT_ARRAY_OFFSET, null, bbAddress + position, countInts * Integer.BYTES); - } else { - unsafe.copyMemory(srcArray, INT_ARRAY_OFFSET, dstByteBuffer.array(), BYTE_ARRAY_OFFSET + position, countInts * Integer.BYTES); - } - } - } From bcac5de3f8b7f0b539cdafd6759fe2a3dba86508 Mon Sep 17 00:00:00 2001 From: Eran Meir Date: Sun, 19 May 2019 11:32:20 +0300 Subject: [PATCH 15/15] updating oak dependency --- processing/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/pom.xml b/processing/pom.xml index 66c007e110cc..99de1faa3097 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -35,7 +35,7 @@ com.yahoo.oak oak - 0.1.4-SNAPSHOT + 0.1.4 org.apache.druid