From 0ecf2be389707717b55cf857ecbea929d4ae3346 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 27 Apr 2018 22:29:25 -0700 Subject: [PATCH] 'shapeshifting' columns, integer encoding --- .../services/org.openjdk.jmh.profile.Profiler | 20 + .../benchmark/BaseColumnarIntsBenchmark.java | 434 ++++++++++++++++++ ...aseColumnarIntsFromGeneratorBenchmark.java | 399 ++++++++++++++++ ...BaseColumnarIntsFromSegmentsBenchmark.java | 265 +++++++++++ ...rIntsEncodeDataFromGeneratorBenchmark.java | 85 ++++ ...narIntsEncodeDataFromSegmentBenchmark.java | 86 ++++ ...rIntsSelectRowsFromGeneratorBenchmark.java | 135 ++++++ ...narIntsSelectRowsFromSegmentBenchmark.java | 140 ++++++ .../druid/benchmark/EncodingSizeProfiler.java | 62 +++ .../benchmark/query/GroupByBenchmark.java | 6 +- pom.xml | 6 + processing/pom.xml | 4 + .../apache/druid/segment/CompressedPools.java | 284 ++++++++++-- .../org/apache/druid/segment/IndexSpec.java | 248 +++++++++- .../segment/StringDimensionMergerV9.java | 46 +- .../druid/segment/column/DoublesColumn.java | 5 + .../druid/segment/column/FloatsColumn.java | 5 + .../druid/segment/column/ValueType.java | 7 + .../data/ColumnarMultiIntsSerializer.java | 4 +- .../CompressedColumnarIntsSerializer.java | 2 +- ...CompressedVSizeColumnarIntsSerializer.java | 2 +- .../CompressedVSizeColumnarIntsSupplier.java | 2 +- .../segment/data/CompressionStrategy.java | 68 ++- .../data/ShapeShiftingBlockColumnarInts.java | 63 +++ .../segment/data/ShapeShiftingColumn.java | 338 ++++++++++++++ .../segment/data/ShapeShiftingColumnData.java | 247 ++++++++++ .../data/ShapeShiftingColumnSerializer.java | 280 +++++++++++ .../data/ShapeShiftingColumnarInts.java | 267 +++++++++++ .../ShapeShiftingColumnarIntsSerializer.java | 198 ++++++++ .../ShapeShiftingColumnarIntsSupplier.java | 201 ++++++++ .../SingleValueColumnarIntsSerializer.java | 4 +- ...essedVSizeColumnarMultiIntsSerializer.java | 2 +- .../data/VSizeColumnarIntsSerializer.java | 2 +- .../VSizeColumnarMultiIntsSerializer.java | 2 +- .../segment/data/codecs/ArrayFormDecoder.java | 26 ++ .../segment/data/codecs/BaseFormDecoder.java | 43 ++ .../segment/data/codecs/BaseFormEncoder.java | 48 ++ .../data/codecs/CompressedFormDecoder.java | 80 ++++ .../data/codecs/CompressedFormEncoder.java | 135 ++++++ .../data/codecs/CompressibleFormEncoder.java | 100 ++++ .../data/codecs/ConstantFormDecoder.java | 26 ++ .../data/codecs/DirectFormDecoder.java | 26 ++ .../segment/data/codecs/FormDecoder.java | 52 +++ .../segment/data/codecs/FormEncoder.java | 99 ++++ .../segment/data/codecs/FormMetrics.java | 91 ++++ .../data/codecs/ints/BaseIntFormEncoder.java | 99 ++++ .../codecs/ints/BytePackedIntFormDecoder.java | 73 +++ .../codecs/ints/BytePackedIntFormEncoder.java | 142 ++++++ .../codecs/ints/CompressedIntFormEncoder.java | 57 +++ .../ints/CompressibleIntFormEncoder.java | 37 ++ .../codecs/ints/ConstantIntFormDecoder.java | 57 +++ .../codecs/ints/ConstantIntFormEncoder.java | 91 ++++ .../segment/data/codecs/ints/IntCodecs.java | 81 ++++ .../data/codecs/ints/IntFormEncoder.java | 26 ++ .../data/codecs/ints/IntFormMetrics.java | 168 +++++++ .../codecs/ints/LemireIntFormDecoder.java | 149 ++++++ .../codecs/ints/LemireIntFormEncoder.java | 120 +++++ .../RunLengthBytePackedIntFormDecoder.java | 283 ++++++++++++ .../RunLengthBytePackedIntFormEncoder.java | 265 +++++++++++ .../codecs/ints/UnencodedIntFormDecoder.java | 53 +++ .../codecs/ints/UnencodedIntFormEncoder.java | 94 ++++ .../data/codecs/ints/ZeroIntFormDecoder.java | 60 +++ .../data/codecs/ints/ZeroIntFormEncoder.java | 75 +++ .../generator/ColumnValueGenerator.java | 7 + .../DictionaryEncodedColumnPartSerde.java | 21 +- .../segment/CustomSegmentizerFactoryTest.java | 4 +- .../apache/druid/segment/IndexSpecTest.java | 31 ++ .../ShapeShiftingColumnarIntsSerdeTest.java | 357 ++++++++++++++ 68 files changed, 6918 insertions(+), 77 deletions(-) create mode 100644 benchmarks/src/main/resources/META-INF/services/org.openjdk.jmh.profile.Profiler create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromGeneratorBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromSegmentsBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromGeneratorBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromSegmentBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromGeneratorBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromSegmentBenchmark.java create mode 100644 benchmarks/src/test/java/org/apache/druid/benchmark/EncodingSizeProfiler.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingBlockColumnarInts.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumn.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnData.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarInts.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSupplier.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ArrayFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/CompressibleFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ConstantFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/DirectFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/FormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/FormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/FormMetrics.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BaseIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressedIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressibleIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntCodecs.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormMetrics.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormEncoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormDecoder.java create mode 100644 processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormEncoder.java create mode 100644 processing/src/test/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerdeTest.java diff --git a/benchmarks/src/main/resources/META-INF/services/org.openjdk.jmh.profile.Profiler b/benchmarks/src/main/resources/META-INF/services/org.openjdk.jmh.profile.Profiler new file mode 100644 index 000000000000..cbe06e9334fa --- /dev/null +++ b/benchmarks/src/main/resources/META-INF/services/org.openjdk.jmh.profile.Profiler @@ -0,0 +1,20 @@ +# +# 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. +# + +org.apache.druid.benchmark.EncodingSizeProfiler diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsBenchmark.java new file mode 100644 index 000000000000..4230fce55e9a --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsBenchmark.java @@ -0,0 +1,434 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.data.ShapeShiftingColumnarIntsSerializer; +import org.apache.druid.segment.data.ShapeShiftingColumnarIntsSupplier; +import org.apache.druid.segment.data.VSizeColumnarInts; +import org.apache.druid.segment.data.codecs.ints.BytePackedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.CompressedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.IntCodecs; +import org.apache.druid.segment.data.codecs.ints.IntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.LemireIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.RunLengthBytePackedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.UnencodedIntFormEncoder; +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.util.BitSet; +import java.util.Map; +import java.util.Random; + +@State(Scope.Benchmark) +public class BaseColumnarIntsBenchmark +{ + static int encodeToFile(int[] vals, int minValue, int maxValue, String encoding, FileChannel output) + throws IOException + { + int numBytes = VSizeColumnarInts.getNumBytesForMax(maxValue); + + ByteOrder byteOrder = ByteOrder.LITTLE_ENDIAN; + + IndexSpec.ShapeShiftBlockSize blockSizeEnum = encoding.endsWith("-13") + ? IndexSpec.ShapeShiftBlockSize.MIDDLE + : encoding.endsWith("-12") + ? IndexSpec.ShapeShiftBlockSize.SMALL + : IndexSpec.ShapeShiftBlockSize.LARGE; + byte blockSize = (byte) (blockSizeEnum.getLogBlockSize() - 2); + IndexSpec.ShapeShiftOptimizationTarget optimizationTarget = + IndexSpec.ShapeShiftOptimizationTarget.FASTBUTSMALLISH; + + + try (SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium()) { + + ByteBuffer uncompressedDataBuffer = + CompressionStrategy.LZ4.getCompressor() + .allocateInBuffer(8 + ((1 << blockSize) * Integer.BYTES), writeOutMedium.getCloser()) + .order(byteOrder); + ByteBuffer compressedDataBuffer = + CompressionStrategy.LZ4.getCompressor() + .allocateOutBuffer( + ((1 << blockSize) * Integer.BYTES) + 1024, + writeOutMedium.getCloser() + ); + switch (encoding) { + case "vsize-byte": + final VSizeColumnarInts vsize = VSizeColumnarInts.fromArray(vals); + vsize.writeTo(output, null); + return (int) vsize.getSerializedSize(); + case "compressed-vsize-byte": + final CompressedVSizeColumnarIntsSupplier compressed = CompressedVSizeColumnarIntsSupplier.fromList( + IntArrayList.wrap(vals), + Math.max(maxValue - 1, 1), + CompressedVSizeColumnarIntsSupplier.maxIntsInBufferForBytes(numBytes), + ByteOrder.nativeOrder(), + CompressionStrategy.LZ4, + Closer.create() + ); + compressed.writeTo(output, null); + return (int) compressed.getSerializedSize(); + case "compressed-vsize-big-endian": + final CompressedVSizeColumnarIntsSupplier compressedBigEndian = CompressedVSizeColumnarIntsSupplier.fromList( + IntArrayList.wrap(vals), + Math.max(maxValue - 1, 1), + CompressedVSizeColumnarIntsSupplier.maxIntsInBufferForBytes(numBytes), + ByteOrder.BIG_ENDIAN, + CompressionStrategy.LZ4, + Closer.create() + ); + compressedBigEndian.writeTo(output, null); + return (int) compressedBigEndian.getSerializedSize(); + case "shapeshift-unencoded": + final IntFormEncoder[] ssucodecs = new IntFormEncoder[]{ + new UnencodedIntFormEncoder( + blockSize, + byteOrder + ) + }; + final ShapeShiftingColumnarIntsSerializer ssunencodedSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + ssucodecs, + optimizationTarget, + blockSizeEnum, + byteOrder + ); + ssunencodedSerializer.open(); + for (int val : vals) { + ssunencodedSerializer.addValue(val); + } + ssunencodedSerializer.writeTo(output, null); + return (int) ssunencodedSerializer.getSerializedSize(); + case "shapeshift-bytepack": + final IntFormEncoder[] ssbytepackcodecs = new IntFormEncoder[]{ + new BytePackedIntFormEncoder( + blockSize, + byteOrder + ) + }; + final ShapeShiftingColumnarIntsSerializer ssbytepackSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + ssbytepackcodecs, + optimizationTarget, + blockSizeEnum, + byteOrder + ); + ssbytepackSerializer.open(); + for (int val : vals) { + ssbytepackSerializer.addValue(val); + } + ssbytepackSerializer.writeTo(output, null); + return (int) ssbytepackSerializer.getSerializedSize(); + case "shapeshift-rle-bytepack": + final IntFormEncoder[] ssrbytepackcodecs = new IntFormEncoder[]{ + new RunLengthBytePackedIntFormEncoder( + blockSize, + byteOrder + ) + }; + final ShapeShiftingColumnarIntsSerializer ssrbytepackSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + ssrbytepackcodecs, + optimizationTarget, + blockSizeEnum, + byteOrder + ); + ssrbytepackSerializer.open(); + for (int val : vals) { + ssrbytepackSerializer.addValue(val); + } + ssrbytepackSerializer.writeTo(output, null); + return (int) ssrbytepackSerializer.getSerializedSize(); + case "shapeshift-lz4-bytepack": + final IntFormEncoder[] sslzcodecs = new IntFormEncoder[]{ + new CompressedIntFormEncoder( + blockSize, + byteOrder, + CompressionStrategy.LZ4, + new BytePackedIntFormEncoder(blockSize, byteOrder), + uncompressedDataBuffer, + compressedDataBuffer + ) + }; + final ShapeShiftingColumnarIntsSerializer sslzSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + sslzcodecs, + optimizationTarget, + blockSizeEnum, + byteOrder + ); + sslzSerializer.open(); + for (int val : vals) { + sslzSerializer.addValue(val); + } + sslzSerializer.writeTo(output, null); + return (int) sslzSerializer.getSerializedSize(); + case "shapeshift-lz4-rle-bytepack": + final IntFormEncoder[] sslzrlecodecs = new IntFormEncoder[]{ + new CompressedIntFormEncoder( + blockSize, + byteOrder, + CompressionStrategy.LZ4, + new RunLengthBytePackedIntFormEncoder(blockSize, byteOrder), + uncompressedDataBuffer, + compressedDataBuffer + ) + }; + final ShapeShiftingColumnarIntsSerializer sslzrleSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + sslzrlecodecs, + optimizationTarget, + blockSizeEnum, + byteOrder + ); + sslzrleSerializer.open(); + for (int val : vals) { + sslzrleSerializer.addValue(val); + } + sslzrleSerializer.writeTo(output, null); + return (int) sslzrleSerializer.getSerializedSize(); + case "shapeshift-fastpfor": + final IntFormEncoder[] dfastcodecs = new IntFormEncoder[]{ + new LemireIntFormEncoder( + blockSize, + IntCodecs.FASTPFOR, + "fastpfor", + byteOrder + ) + }; + final ShapeShiftingColumnarIntsSerializer ssfastPforSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + dfastcodecs, + optimizationTarget, + blockSizeEnum, + byteOrder + ); + ssfastPforSerializer.open(); + for (int val : vals) { + ssfastPforSerializer.addValue(val); + } + ssfastPforSerializer.writeTo(output, null); + return (int) ssfastPforSerializer.getSerializedSize(); + case "shapeshift": + case "shapeshift-13": + case "shapeshift-12": + case "shapeshift-lazy": + case "shapeshift-eager": + case "shapeshift-faster": + case "shapeshift-faster-13": + case "shapeshift-faster-12": + case "shapeshift-smaller": + case "shapeshift-smaller-13": + case "shapeshift-smaller-12": + final IntFormEncoder[] sscodecs = ShapeShiftingColumnarIntsSerializer.getDefaultIntFormEncoders( + blockSizeEnum, + CompressionStrategy.LZ4, + writeOutMedium.getCloser(), + byteOrder + ); + final ShapeShiftingColumnarIntsSerializer ssSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + sscodecs, + encoding.contains("shapeshift-smaller") + ? IndexSpec.ShapeShiftOptimizationTarget.SMALLER + : encoding.contains("shapeshift-faster") + ? IndexSpec.ShapeShiftOptimizationTarget.FASTER + : optimizationTarget, + blockSizeEnum, + byteOrder + ); + ssSerializer.open(); + for (int val : vals) { + ssSerializer.addValue(val); + } + ssSerializer.writeTo(output, null); + return (int) ssSerializer.getSerializedSize(); + case "shapeshift-lz4-only": + final IntFormEncoder[] sslzNewcodecs = new IntFormEncoder[]{ + new CompressedIntFormEncoder( + blockSize, + byteOrder, + CompressionStrategy.LZ4, + new RunLengthBytePackedIntFormEncoder(blockSize, byteOrder), + uncompressedDataBuffer, + compressedDataBuffer + ), + new CompressedIntFormEncoder( + blockSize, + byteOrder, + CompressionStrategy.LZ4, + new BytePackedIntFormEncoder(blockSize, byteOrder), + uncompressedDataBuffer, + compressedDataBuffer + ), + }; + final ShapeShiftingColumnarIntsSerializer sslzNewSerializer = + new ShapeShiftingColumnarIntsSerializer( + writeOutMedium, + sslzNewcodecs, + optimizationTarget, + blockSizeEnum, + byteOrder + ); + sslzNewSerializer.open(); + for (int val : vals) { + sslzNewSerializer.addValue(val); + } + sslzNewSerializer.writeTo(output, null); + return (int) sslzNewSerializer.getSerializedSize(); + } + throw new IllegalArgumentException("unknown encoding"); + } + } + + static ColumnarInts createIndexedInts(String encoding, ByteBuffer buffer, int size) + { + ByteOrder byteOrder = ByteOrder.LITTLE_ENDIAN; + switch (encoding) { + case "vsize-byte": + return VSizeColumnarInts.readFromByteBuffer(buffer); + case "compressed-vsize-byte": + return CompressedVSizeColumnarIntsSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder()).get(); + case "compressed-vsize-big-endian": + return CompressedVSizeColumnarIntsSupplier.fromByteBuffer(buffer, ByteOrder.BIG_ENDIAN).get(); + case "shapeshift": + case "shapeshift-unencoded": + case "shapeshift-fastpfor": + case "shapeshift-bytepack": + case "shapeshift-lz4-bytepack": + case "shapeshift-rle-bytepack": + case "shapeshift-lz4-rle-bytepack": + case "shapeshift-lz4-only": + case "shapeshift-smaller": + case "shapeshift-faster": + case "shapeshift-13": + case "shapeshift-smaller-13": + case "shapeshift-faster-13": + case "shapeshift-12": + case "shapeshift-smaller-12": + case "shapeshift-faster-12": + return ShapeShiftingColumnarIntsSupplier.fromByteBuffer(buffer, byteOrder).get(); + case "shapeshift-lazy": + return ShapeShiftingColumnarIntsSupplier.fromByteBuffer( + buffer, + byteOrder, + ShapeShiftingColumnarIntsSupplier.ShapeShiftingColumnarIntsDecodeOptimization.MIXED + ).get(); + case "shapeshift-eager": + return ShapeShiftingColumnarIntsSupplier.fromByteBuffer( + buffer, + byteOrder, + ShapeShiftingColumnarIntsSupplier.ShapeShiftingColumnarIntsDecodeOptimization.BLOCK + ).get(); + } + throw new IllegalArgumentException("unknown encoding"); + } + + // for debugging: validate that all encoders read the same values + static void checkSanity(Map encoders, ImmutableList encodings, int rows) + throws Exception + { + for (int i = 0; i < rows; i++) { + checkRowSanity(encoders, encodings, i); + } + } + + static void checkRowSanity(Map encoders, ImmutableList encodings, int row) + throws Exception + { + if (encodings.size() > 1) { + for (int i = 0; i < encodings.size() - 1; i++) { + String currentKey = encodings.get(i); + String nextKey = encodings.get(i + 1); + IndexedInts current = encoders.get(currentKey); + IndexedInts next = encoders.get(nextKey); + int vCurrent = current.get(row); + int vNext = next.get(row); + if (vCurrent != vNext) { + throw new Exception("values do not match at row " + + row + + " - " + + currentKey + + ":" + + vCurrent + + " " + + nextKey + + ":" + + vNext); + } + } + } + } + + //@Param({"shapeshift-bytepack", "shapeshift-rle-bytepack", "shapeshift-fastpfor", "shapeshift-lz4-bytepack", "shapeshift-lz4-rle-bytepack", "compressed-vsize-byte"}) + @Param({"compressed-vsize-byte", "shapeshift"}) + String encoding; + + Random rand = new Random(0); + + int[] vals; + + int minValue; + int maxValue; + BitSet filter; + + void setupFilters(int rows, double filteredRowCountPercetnage) + { + // todo: save and read from file for stable filter set.. + // todo: also maybe filter set distributions to simulate different select patterns? + // (because benchmarks don't take long enough already..) + filter = null; + final int filteredRowCount = (int) Math.floor(rows * filteredRowCountPercetnage); + + if (filteredRowCount < rows) { + // setup bitset filter + filter = new BitSet(); + for (int i = 0; i < filteredRowCount; i++) { + int rowToAccess = rand.nextInt(rows); + // Skip already selected rows if any + while (filter.get(rowToAccess)) { + rowToAccess = (rowToAccess + 1) % rows; + } + filter.set(rowToAccess); + } + } + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromGeneratorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromGeneratorBenchmark.java new file mode 100644 index 000000000000..ce74ec2c2189 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromGeneratorBenchmark.java @@ -0,0 +1,399 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import io.netty.util.SuppressForbidden; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.generator.ColumnValueGenerator; +import org.apache.druid.segment.generator.GeneratorColumnSchema; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; + +@State(Scope.Benchmark) +public class BaseColumnarIntsFromGeneratorBenchmark extends BaseColumnarIntsBenchmark +{ + static ColumnValueGenerator makeGenerator( + String distribution, + int bits, + int bound, + int cardinality, + int rows + ) + { + switch (distribution) { + case "enumerated": + ImmutableList enumerated; + ImmutableList probability; + + switch (bits) { + case 1: + enumerated = ImmutableList.of(0, 1); + probability = ImmutableList.of(0.95, 0.001); + break; + case 2: + enumerated = ImmutableList.of(0, 1, 2, 3); + probability = ImmutableList.of(0.95, 0.001, 0.0189, 0.03); + break; + default: + enumerated = ImmutableList.of(0, 1, bound / 4, bound / 2, 3 * bound / 4, bound); + probability = ImmutableList.of(0.90, 0.001, 0.0189, 0.03, 0.0001, 0.025); + break; + } + GeneratorColumnSchema enumeratedSchema = GeneratorColumnSchema.makeEnumerated( + "", + ValueType.INT, + true, + 1, + 0d, + enumerated, + probability + ); + return enumeratedSchema.makeGenerator(1); + case "zipfLow": + GeneratorColumnSchema zipfLowSchema = GeneratorColumnSchema.makeZipf( + "", + ValueType.INT, + true, + 1, + 0d, + Math.max(bound - cardinality, 0), + bound, + 1d + ); + return zipfLowSchema.makeGenerator(1); + case "lazyZipfLow": + GeneratorColumnSchema lzipfLowSchema = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0d, + Math.max(bound - cardinality, 0), + bound, + 1d + ); + return lzipfLowSchema.makeGenerator(1); + case "zipfHi": + GeneratorColumnSchema zipfHighSchema = GeneratorColumnSchema.makeZipf( + "", + ValueType.INT, + true, + 1, + 0d, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return zipfHighSchema.makeGenerator(1); + case "lazyZipfHi": + GeneratorColumnSchema lzipfHighSchema = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0d, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return lzipfHighSchema.makeGenerator(1); + case "nullp50zipfLow": + GeneratorColumnSchema nullp50ZipfLow = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.5, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return nullp50ZipfLow.makeGenerator(1); + case "nullp75zipfLow": + GeneratorColumnSchema nullp75ZipfLow = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.75, + Math.max(bound - cardinality, 0), + bound, + 1d + ); + return nullp75ZipfLow.makeGenerator(1); + case "nullp90zipfLow": + GeneratorColumnSchema nullp90ZipfLow = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.9, + Math.max(bound - cardinality, 0), + bound, + 1d + ); + return nullp90ZipfLow.makeGenerator(1); + case "nullp95zipfLow": + GeneratorColumnSchema nullp95ZipfLow = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.95, + Math.max(bound - cardinality, 0), + bound, + 1d + ); + return nullp95ZipfLow.makeGenerator(1); + case "nullp99zipfLow": + GeneratorColumnSchema nullp99ZipfLow = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.99, + Math.max(bound - cardinality, 0), + bound, + 1d + ); + return nullp99ZipfLow.makeGenerator(1); + case "nullp50zipfHi": + GeneratorColumnSchema nullp50ZipfHi = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.5, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return nullp50ZipfHi.makeGenerator(1); + case "nullp75zipfHi": + GeneratorColumnSchema nullp75ZipfHi = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.75, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return nullp75ZipfHi.makeGenerator(1); + case "nullp90zipfHi": + GeneratorColumnSchema nullp90ZipfHi = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.9, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return nullp90ZipfHi.makeGenerator(1); + case "nullp95zipfHi": + GeneratorColumnSchema nullp95ZipfHi = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.95, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return nullp95ZipfHi.makeGenerator(1); + case "nullp99zipfHi": + GeneratorColumnSchema nullp99ZipfHi = GeneratorColumnSchema.makeLazyZipf( + "", + ValueType.INT, + true, + 1, + 0.99, + Math.max(bound - cardinality, 0), + bound, + 3d + ); + return nullp99ZipfHi.makeGenerator(1); + case "sequential": + GeneratorColumnSchema sequentialSchema = GeneratorColumnSchema.makeSequential( + "", + ValueType.INT, + true, + 1, + 0d, + Math.max(bound - rows, 0), + bound + ); + return sequentialSchema.makeGenerator(1); + case "sequential-skip": + GeneratorColumnSchema sequentialSkipSchema = GeneratorColumnSchema.makeSequential( + "", + ValueType.INT, + true, + 1, + 0d, + 0, + bound + ); + return sequentialSkipSchema.makeGenerator(1); + case "uniform": + GeneratorColumnSchema uniformSchema = GeneratorColumnSchema.makeDiscreteUniform( + "", + ValueType.INT, + true, + 1, + 0d, + Math.max(bound - cardinality, 0), + bound + ); + return uniformSchema.makeGenerator(1); + case "lazyUniform": + GeneratorColumnSchema lazyUniformSchema = GeneratorColumnSchema.makeLazyDiscreteUniform( + "", + ValueType.INT, + true, + 1, + 0d, + Math.max(bound - cardinality, 0), + bound + ); + return lazyUniformSchema.makeGenerator(1); + } + throw new IllegalArgumentException("unknown distribution"); + } + + //@Param({"3", "9", "18", "27"}) + @Param({"27"}) + int bits; + + @Param({"3000000"}) + int rows; + + //@Param({"lazyZipfLow", "lazyZipfHi", "nullp50zipfLow", "nullp75zipfLow", "nullp90zipfLow", "nullp95zipfLow", "nullp99zipfLow", "lazyUniform", "random"}) + @Param({"nullp95zipfHi"}) + String distribution; + + @Param("150000000") + int cardinality; + + int bound; + + @SuppressForbidden(reason = "System#out") + void initializeValues() throws IOException + { + final String filename = getGeneratorValueFilename(distribution, cardinality, bits, rows); + File dir = getTmpDir(); + File dataFile = new File(dir, filename); + + vals = new int[rows]; + bound = 1 << bits; + + if (dataFile.exists()) { + System.out.println("Data files already exist, re-using\n"); + try (BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8)) { + int lineNum = 0; + String line; + while ((line = br.readLine()) != null) { + vals[lineNum] = Integer.parseInt(line); + if (vals[lineNum] < minValue) { + minValue = vals[lineNum]; + } + if (vals[lineNum] > maxValue) { + maxValue = vals[lineNum]; + } + lineNum++; + } + } + } else { + try (Writer writer = Files.newBufferedWriter(dataFile.toPath(), StandardCharsets.UTF_8)) { + int atLeastOneMustBeAsLargeAsBound = rand.nextInt(rows); + if ("random".equals(distribution)) { + for (int i = 0; i < vals.length; ++i) { + vals[i] = rand.nextInt(bound); + if (i == atLeastOneMustBeAsLargeAsBound) { + vals[i] = bound; + } + if (vals[i] < minValue) { + minValue = vals[i]; + } + if (vals[i] > maxValue) { + maxValue = vals[i]; + } + writer.write(vals[i] + "\n"); + } + } else { + ColumnValueGenerator valueGenerator = makeGenerator(distribution, bits, bound, cardinality, rows); + + for (int i = 0; i < vals.length; ++i) { + int value; + Object rowValue = valueGenerator.generateRowValue(); + value = rowValue != null ? (int) rowValue : 0; + if (i == atLeastOneMustBeAsLargeAsBound) { + value = bound; + } else if ("sequential-skip".equals(distribution) && bits > 1) { + int skip = Math.max(bound / cardinality, 1); + for (int burn = 0; burn < skip; burn++) { + value = (int) valueGenerator.generateRowValue(); + } + } + vals[i] = value; + if (vals[i] < minValue) { + minValue = vals[i]; + } + if (vals[i] > maxValue) { + maxValue = vals[i]; + } + writer.write(vals[i] + "\n"); + } + } + } + } + } + + static String getGeneratorValueFilename(String distribution, int cardinality, int bits, int rows) + { + return "values-" + distribution + "-" + cardinality + "-" + bits + "-" + rows + ".bin"; + } + + static String getGeneratorEncodedFilename(String encoding, int bits, String distribution, int rows, int cardinality) + { + return encoding + "-" + bits + "-" + distribution + "-" + rows + "-" + cardinality + ".bin"; + } + + static File getTmpDir() + { + final String dirPath = "tmp/encoding/ints/"; + File dir = new File(dirPath); + dir.mkdirs(); + return dir; + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromSegmentsBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromSegmentsBenchmark.java new file mode 100644 index 000000000000..52ba1b2826a4 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/BaseColumnarIntsFromSegmentsBenchmark.java @@ -0,0 +1,265 @@ +/* + * 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; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Iterables; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.DictionaryEncodedColumn; +import org.apache.druid.segment.column.ValueType; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; + +import java.io.BufferedReader; +import java.io.File; +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Set; + +@State(Scope.Benchmark) +public class BaseColumnarIntsFromSegmentsBenchmark extends BaseColumnarIntsBenchmark +{ + //CHECKSTYLE.OFF: Regexp + // wiki columns + @Param({ + "channel", + "cityName", + "comment", + "commentLength", + "countryIsoCode", + "countryName", + "deltaBucket", + "diffUrl", + "flags", + "isAnonymous", + "isMinor", + "isNew", + "isRobot", + "isUnpatrolled", + "metroCode", + "namespace", + "page", + "regionIsoCode", + "regionName", + "user" + }) + + // twitter columns +// @Param({ +// "geo", +// "lang", +// "retweet", +// "screen_name", +// "source", +// "text", +// "utc_offset", +// "verified" +// }) + + // clarity columns +// @Param({ +// "bufferpoolName", +// "clarityTopic", +// "clarityUser", +// "context", +// "dataSource", +// "description", +// "dimension", +// "duration", +// "feed", +// "gcGen", +// "gcGenSpaceName", +// "gcName", +// "hasFilters", +// "host", +// "id", +// "identity", +// "implyCluster", +// "implyDruidVersion", +// "implyNodeType", +// "implyVersion", +// "memKind", +// "memcached txt", +// "metric", +// "numComplexMetrics", +// "numDimensions", +// "numMetrics", +// "poolKind", +// "poolName", +// "priority", +// "remoteAddr", +// "remoteAddress", +// "server", +// "service", +// "severity", +// "success", +// "taskId", +// "taskStatus", +// "taskType", +// "threshold", +// "tier", +// "type", +// "version" +// }) + + // lineitem columns +// @Param({ +// "l_comment", +// "l_commitdate", +// "l_linenumber", +// "l_linestatus", +// "l_orderkey", +// "l_partkey", +// "l_receiptdate", +// "l_returnflag", +// "l_shipinstruct", +// "l_shipmode", +// "l_suppkey" +// }) + String columnName; + +// @Param({"533652"}) // wiki + @Param({"3537476"}) // wiki-2 +// @Param({"3259585"}) // twitter +// @Param({"3783642"}) // clarity +// @Param({"6001215"}) // tpch-lineitem-1g + int rows; + + +// @Param({"tmp/segments/wiki-1/"}) + @Param({"tmp/segments/wiki-2/"}) +// @Param({"tmp/segments/twitter-1/"}) +// @Param({"tmp/segments/clarity-1/"}) +// @Param({"tmp/segments/tpch-lineitem-1/"}) + String segmentPath; + +// @Param({"wikiticker"}) + @Param({"wikiticker-2"}) +// @Param({"twitter"}) +// @Param({"clarity"}) +// @Param({"tpch-lineitem"}) + String segmentName; + + + private static IndexIO INDEX_IO; + public static ObjectMapper JSON_MAPPER; + + //CHECKSTYLE.ON: Regexp + + /** + * read column intermediary values into integer array + * + * @throws IOException + */ + void initializeValues() throws IOException + { + initializeSegmentValueIntermediaryFile(); + File dir = getTmpDir(); + File dataFile = new File(dir, getColumnDataFileName(segmentName, columnName)); + + ArrayList values = new ArrayList<>(); + try (BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8)) { + String line; + while ((line = br.readLine()) != null) { + int value = Integer.parseInt(line); + if (value < minValue) { + minValue = value; + } + if (value > maxValue) { + maxValue = value; + } + values.add(value); + rows++; + } + } + + vals = values.stream().mapToInt(i -> i).toArray(); + } + + String getColumnDataFileName(String segmentName, String columnName) + { + return StringUtils.format("%s-ints-%s.txt", segmentName, columnName); + } + + String getColumnEncodedFileName(String encoding, String segmentName, String columnName) + { + return StringUtils.format("%s-%s-ints-%s.bin", encoding, segmentName, columnName); + } + + File getTmpDir() + { + final String dirPath = StringUtils.format("tmp/encoding/%s", segmentName); + File dir = new File(dirPath); + dir.mkdirs(); + return dir; + } + + /** + * writes column values to text file, 1 per line + * + * @throws IOException + */ + void initializeSegmentValueIntermediaryFile() throws IOException + { + File dir = getTmpDir(); + File dataFile = new File(dir, getColumnDataFileName(segmentName, columnName)); + + if (!dataFile.exists()) { + JSON_MAPPER = new DefaultObjectMapper(); + INDEX_IO = new IndexIO( + JSON_MAPPER, + () -> 0 + ); + try (final QueryableIndex index = INDEX_IO.loadIndex(new File(segmentPath))) { + final Set columnNames = new HashSet<>(); + columnNames.add(ColumnHolder.TIME_COLUMN_NAME); + Iterables.addAll(columnNames, index.getColumnNames()); + final ColumnHolder column = index.getColumnHolder(columnName); + final ColumnCapabilities capabilities = column.getCapabilities(); + final ValueType columnType = capabilities.getType(); + try (Writer writer = Files.newBufferedWriter(dataFile.toPath(), StandardCharsets.UTF_8)) { + if (columnType != ValueType.STRING) { + throw new RuntimeException("Invalid column type, expected 'String'"); + } + DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) column.getColumn(); + + if (theColumn.hasMultipleValues()) { + throw new RuntimeException("Multi-int benchmarks are not current supported"); + } + + for (int i = 0; i < theColumn.length(); i++) { + int value = theColumn.getSingleValueRow(i); + writer.write(value + "\n"); + } + } + } + } + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromGeneratorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromGeneratorBenchmark.java new file mode 100644 index 000000000000..adca25aa0fb4 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromGeneratorBenchmark.java @@ -0,0 +1,85 @@ +/* + * 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; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 1) +@Measurement(iterations = 1) +public class ColumnarIntsEncodeDataFromGeneratorBenchmark extends BaseColumnarIntsFromGeneratorBenchmark +{ + @Setup + public void setup() throws Exception + { + initializeValues(); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void encodeColumn(Blackhole blackhole) throws IOException + { + File dir = getTmpDir(); + File columnDataFile = new File(dir, getGeneratorEncodedFilename(encoding, bits, distribution, rows, cardinality)); + columnDataFile.delete(); + FileChannel output = + FileChannel.open(columnDataFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE); + + int size = encodeToFile(vals, minValue, maxValue, encoding, output); + EncodingSizeProfiler.encodedSize = size; + blackhole.consume(size); + output.close(); + } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(ColumnarIntsEncodeDataFromGeneratorBenchmark.class.getSimpleName()) + .addProfiler(EncodingSizeProfiler.class) + .resultFormat(ResultFormatType.CSV) + .result("column-ints-encode-speed.csv") + .build(); + + new Runner(opt).run(); + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromSegmentBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromSegmentBenchmark.java new file mode 100644 index 000000000000..b5a6366ee06d --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsEncodeDataFromSegmentBenchmark.java @@ -0,0 +1,86 @@ +/* + * 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; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 1) +@Measurement(iterations = 3) +public class ColumnarIntsEncodeDataFromSegmentBenchmark extends BaseColumnarIntsFromSegmentsBenchmark +{ + @Setup + public void setup() throws Exception + { + initializeValues(); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void encodeColumn(Blackhole blackhole) throws IOException + { + File dir = getTmpDir(); + File columnDataFile = new File(dir, getColumnEncodedFileName(encoding, segmentName, columnName)); + columnDataFile.delete(); + FileChannel output = + FileChannel.open(columnDataFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE); + + int size = BaseColumnarIntsBenchmark.encodeToFile(vals, minValue, maxValue, encoding, output); + EncodingSizeProfiler.encodedSize = size; + blackhole.consume(size); + output.close(); + } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(ColumnarIntsEncodeDataFromSegmentBenchmark.class.getSimpleName()) + .addProfiler(EncodingSizeProfiler.class) + .resultFormat(ResultFormatType.CSV) + .result("column-ints-encode-speed-segments.csv") + .build(); + + new Runner(opt).run(); + } +} + diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromGeneratorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromGeneratorBenchmark.java new file mode 100644 index 000000000000..da9bfb9f8682 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromGeneratorBenchmark.java @@ -0,0 +1,135 @@ +/* + * 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; + +import com.google.common.io.Files; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.IndexedInts; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +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 org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 5) +public class ColumnarIntsSelectRowsFromGeneratorBenchmark extends BaseColumnarIntsFromGeneratorBenchmark +{ + private Map encoders; + private Map encodedSize; + + // Number of rows to read, the test will read random rows + @Param({"0.01", "0.1", "0.33", "0.66", "0.95", "1.0"}) + private double filteredRowCountPercentage; + + @Setup + public void setup() throws Exception + { + encoders = new HashMap<>(); + encodedSize = new HashMap<>(); + + setupFilters(rows, filteredRowCountPercentage); + setupFromFile(encoding); + + // uncomment me to load multiple encoded files for sanity check + //CHECKSTYLE.OFF: Regexp +// ImmutableList all = ImmutableList.of("compressed-vsize-byte", "shapeshift-fastpfor"); +// for (String _enc : all) { +// if (!_enc.equalsIgnoreCase(encoding)) { +// setupFromFile(_enc); +// } +// } +// +// checkSanity(encoders, all, rows); + //CHECKSTYLE.ON: Regexp + } + + @TearDown + public void teardown() throws Exception + { + for (ColumnarInts ints : encoders.values()) { + ints.close(); + } + } + + private void setupFromFile(String encoding) throws IOException + { + File dir = getTmpDir(); + File compFile = new File(dir, getGeneratorEncodedFilename(encoding, bits, distribution, rows, cardinality)); + ByteBuffer buffer = Files.map(compFile); + + int size = (int) compFile.length(); + encodedSize.put(encoding, size); + ColumnarInts data = createIndexedInts(encoding, buffer, size); + encoders.put(encoding, data); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void selectRows(Blackhole blackhole) + { + EncodingSizeProfiler.encodedSize = encodedSize.get(encoding); + IndexedInts encoder = encoders.get(encoding); + if (filter == null) { + for (int i = 0; i < rows; i++) { + blackhole.consume(encoder.get(i)); + } + } else { + for (int i = filter.nextSetBit(0); i >= 0; i = filter.nextSetBit(i + 1)) { + blackhole.consume(encoder.get(i)); + } + } + } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(ColumnarIntsSelectRowsFromGeneratorBenchmark.class.getSimpleName()) + .addProfiler(EncodingSizeProfiler.class) + .resultFormat(ResultFormatType.CSV) + .result("column-ints-select-speed.csv") + .build(); + + new Runner(opt).run(); + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromSegmentBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromSegmentBenchmark.java new file mode 100644 index 000000000000..ec83596bf08c --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ColumnarIntsSelectRowsFromSegmentBenchmark.java @@ -0,0 +1,140 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import com.google.common.io.Files; +import org.apache.druid.segment.data.ColumnarInts; +import org.apache.druid.segment.data.IndexedInts; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +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 org.openjdk.jmh.results.format.ResultFormatType; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.RunnerException; +import org.openjdk.jmh.runner.options.Options; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 5) +@Measurement(iterations = 10) +public class ColumnarIntsSelectRowsFromSegmentBenchmark extends BaseColumnarIntsFromSegmentsBenchmark +{ + private Map encoders; + private Map encodedSize; + + // Number of rows to read, the test will read random rows + @Param({"0.01", "0.1", "0.33", "0.66", "0.95", "1.0"}) + private double filteredRowCountPercentage; + + Random rand = new Random(0); + + @Setup + public void setup() throws Exception + { + encoders = new HashMap<>(); + encodedSize = new HashMap<>(); + setupFilters(rows, filteredRowCountPercentage); + + setupFromFile(encoding); + + + // uncomment me to load some encoding files to cross reference values for sanity check + //CHECKSTYLE.OFF: Regexp + ImmutableList all = ImmutableList.of("compressed-vsize-byte", "shapeshift"); + for (String _enc : all) { + if (!_enc.equals(encoding)) { + setupFromFile(_enc); + } + } + checkSanity(encoders, all, rows); + //CHECKSTYLE.ON: Regexp + } + + @TearDown + public void teardown() throws Exception + { + for (ColumnarInts ints : encoders.values()) { + ints.close(); + } + } + + private void setupFromFile(String encoding) throws IOException + { + File dir = getTmpDir(); + File compFile = new File(dir, getColumnEncodedFileName(encoding, segmentName, columnName)); + ByteBuffer buffer = Files.map(compFile); + + int size = (int) compFile.length(); + encodedSize.put(encoding, size); + ColumnarInts data = BaseColumnarIntsBenchmark.createIndexedInts(encoding, buffer, size); + encoders.put(encoding, data); + } + + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void selectRows(Blackhole blackhole) + { + EncodingSizeProfiler.encodedSize = encodedSize.get(encoding); + IndexedInts encoder = encoders.get(encoding); + if (filter == null) { + for (int i = 0; i < rows; i++) { + blackhole.consume(encoder.get(i)); + } + } else { + for (int i = filter.nextSetBit(0); i >= 0; i = filter.nextSetBit(i + 1)) { + blackhole.consume(encoder.get(i)); + } + } + } + + public static void main(String[] args) throws RunnerException + { + Options opt = new OptionsBuilder() + .include(ColumnarIntsSelectRowsFromSegmentBenchmark.class.getSimpleName()) + .addProfiler(EncodingSizeProfiler.class) + .resultFormat(ResultFormatType.CSV) + .result("column-ints-select-speed-segments.csv") + .build(); + + new Runner(opt).run(); + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/EncodingSizeProfiler.java b/benchmarks/src/test/java/org/apache/druid/benchmark/EncodingSizeProfiler.java new file mode 100644 index 000000000000..cc9aa0eb4cce --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/EncodingSizeProfiler.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.benchmark; + +import org.openjdk.jmh.infra.BenchmarkParams; +import org.openjdk.jmh.infra.IterationParams; +import org.openjdk.jmh.profile.InternalProfiler; +import org.openjdk.jmh.results.AggregationPolicy; +import org.openjdk.jmh.results.IterationResult; +import org.openjdk.jmh.results.Result; +import org.openjdk.jmh.results.ScalarResult; + +import java.util.Collection; +import java.util.Collections; + +public class EncodingSizeProfiler implements InternalProfiler +{ + public static int encodedSize; + + @Override + public void beforeIteration( + BenchmarkParams benchmarkParams, + IterationParams iterationParams + ) + { + } + + @Override + public Collection afterIteration( + BenchmarkParams benchmarkParams, + IterationParams iterationParams, + IterationResult result + ) + { + return Collections.singletonList( + new ScalarResult("encoded size", (double) encodedSize, "bytes", AggregationPolicy.MAX) + ); + } + + @Override + public String getDescription() + { + return "super janky encoding size result collector"; + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java index 57214154cd2c..24beb990810a 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java @@ -124,7 +124,7 @@ public class GroupByBenchmark @Param({"4"}) private int numSegments; - @Param({"2", "4"}) + @Param({"2"}) private int numProcessingThreads; @Param({"-1"}) @@ -133,10 +133,10 @@ public class GroupByBenchmark @Param({"100000"}) private int rowsPerSegment; - @Param({"basic.A", "basic.nested"}) + @Param({"basic.nested"}) private String schemaAndQuery; - @Param({"v1", "v2"}) + @Param({"v2"}) private String defaultStrategy; @Param({"all", "day"}) diff --git a/pom.xml b/pom.xml index 8c024bd83cba..eba8314d0d46 100644 --- a/pom.xml +++ b/pom.xml @@ -920,6 +920,12 @@ RoaringBitmap 0.8.11 + + me.lemire.integercompression + JavaFastPFOR + 0.1.11 + + org.ow2.asm asm diff --git a/processing/pom.xml b/processing/pom.xml index 5355bf1a16a7..4707d8362306 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -75,6 +75,10 @@ org.roaringbitmap RoaringBitmap + + me.lemire.integercompression + JavaFastPFOR + it.unimi.dsi fastutil diff --git a/processing/src/main/java/org/apache/druid/segment/CompressedPools.java b/processing/src/main/java/org/apache/druid/segment/CompressedPools.java index d62443270bb5..3cbccf2cef5e 100644 --- a/processing/src/main/java/org/apache/druid/segment/CompressedPools.java +++ b/processing/src/main/java/org/apache/druid/segment/CompressedPools.java @@ -20,11 +20,18 @@ package org.apache.druid.segment; import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import com.ning.compress.BufferRecycler; +import me.lemire.integercompression.FastPFOR; +import me.lemire.integercompression.SkippableComposition; +import me.lemire.integercompression.SkippableIntegerCODEC; +import me.lemire.integercompression.VariableByte; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ResourceHolder; import org.apache.druid.collections.StupidPool; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.ints.IntCodecs; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -34,7 +41,26 @@ public class CompressedPools { private static final Logger log = new Logger(CompressedPools.class); + private static final int SMALLEST_BUFFER_SIZE = 0x4000; + private static final int SMALLER_BUFFER_SIZE = 0x8000; public static final int BUFFER_SIZE = 0x10000; + // Straight from the horse's mouth (https://github.com/lemire/JavaFastPFOR/blob/master/example.java). + private static final int ENCODED_INTS_SHOULD_BE_ENOUGH = 1024; + private static final int INT_ARRAY_SIZE = 1 << 14; + private static final int SMALLER_INT_ARRAY_SIZE = 1 << 13; + private static final int SMALLEST_INT_ARRAY_SIZE = 1 << 12; + + // todo: i have no idea what these should legitimately be, this is only ~24.7M which cannot be reclaimed by gc... + // ...but maybe convservative if there is a lot of load, perhaps this is configurable? + private static final int INT_DECODED_ARRAY_POOL_MAX_CACHE = 256; + private static final int INT_ENCODED_ARRAY_POOL_MAX_CACHE = 128; + + + // todo: see ^ re: sizing.. these are currently ~1M on heap + ~200K direct buffer. Heap could be ~1/4 of the size + // with minor changes to fastpfor lib to allow passing page size (our max is 2^14 but codec allocates for 2^16) + // current sizing put it in around 33.6M that cannot be reclaimed + private static final int LEMIRE_FASTPFOR_CODEC_POOL_MAX_CACHE = 28; + private static final NonBlockingPool BUFFER_RECYCLER_POOL = new StupidPool<>( "bufferRecyclerPool", new Supplier() @@ -75,41 +101,245 @@ public static ResourceHolder getOutputBytes() return OUTPUT_BYTES_POOL.take(); } - private static final NonBlockingPool BIG_ENDIAN_BYTE_BUF_POOL = new StupidPool( - "bigEndByteBufPool", - new Supplier() - { - private final AtomicLong counter = new AtomicLong(0); - - @Override - public ByteBuffer get() + private static NonBlockingPool makeBufferPool(String name, int size, ByteOrder order) + { + return new StupidPool<>( + name, + new Supplier() { - log.debug("Allocating new bigEndByteBuf[%,d]", counter.incrementAndGet()); - return ByteBuffer.allocateDirect(BUFFER_SIZE).order(ByteOrder.BIG_ENDIAN); + private final AtomicLong counter = new AtomicLong(0); + + @Override + public ByteBuffer get() + { + log.info("Allocating new %s[%,d]", name, counter.incrementAndGet()); + return ByteBuffer.allocateDirect(size).order(order); + } } - } - ); + ); + } - private static final NonBlockingPool LITTLE_ENDIAN_BYTE_BUF_POOL = new StupidPool( - "littleEndByteBufPool", - new Supplier() - { - private final AtomicLong counter = new AtomicLong(0); + private static NonBlockingPool makeIntArrayPool(String name, int size, int maxCache) + { + return new StupidPool<>( + name, + new Supplier() + { + private final AtomicLong counter = new AtomicLong(0); - @Override - public ByteBuffer get() + @Override + public int[] get() + { + log.info("Allocating new %s[%,d]", name, counter.incrementAndGet()); + return new int[size]; + } + }, + 0, + maxCache + ); + } + + private static NonBlockingPool makeFastpforPool(String name, int size) + { + return new StupidPool<>( + name, + new Supplier() { - log.debug("Allocating new littleEndByteBuf[%,d]", counter.incrementAndGet()); - return ByteBuffer.allocateDirect(BUFFER_SIZE).order(ByteOrder.LITTLE_ENDIAN); - } - } - ); + private final AtomicLong counter = new AtomicLong(0); + + @Override + public SkippableIntegerCODEC get() + { + log.info("Allocating new %s[%,d]", name, counter.incrementAndGet()); + + Supplier compressionBufferSupplier = + Suppliers.memoize(() -> ByteBuffer.allocateDirect(size)); + return new SkippableComposition( + new FastPFOR(), + new VariableByte() { + // VariableByte allocates a buffer in compress method instead of in constructor like fastpfor + // so override to re-use instead (and only allocate if indexing) + @Override + protected ByteBuffer makeBuffer(int sizeInBytes) + { + ByteBuffer theBuffer = compressionBufferSupplier.get(); + theBuffer.clear(); + return theBuffer; + } + } + ); + } + }, + 0, + LEMIRE_FASTPFOR_CODEC_POOL_MAX_CACHE + ); + } + + private static final NonBlockingPool BIG_END_BYTE_BUF_POOL = + makeBufferPool("bigEndByteBufPool", BUFFER_SIZE, ByteOrder.BIG_ENDIAN); + + private static final NonBlockingPool LITTLE_BIG_END_BYTE_BUF_POOL = + makeBufferPool("littleBigEndByteBufPool", SMALLER_BUFFER_SIZE, ByteOrder.BIG_ENDIAN); + + private static final NonBlockingPool LITTLEST_BIG_END_BYTE_BUF_POOL = + makeBufferPool("littlestBigEndByteBufPool", SMALLEST_BUFFER_SIZE, ByteOrder.BIG_ENDIAN); + + private static final NonBlockingPool LITTLE_END_BYTE_BUF_POOL = + makeBufferPool("littleEndByteBufPool", BUFFER_SIZE, ByteOrder.LITTLE_ENDIAN); + + private static final NonBlockingPool LITTLER_END_BYTE_BUF_POOL = + makeBufferPool("littlerEndByteBufPool", SMALLER_BUFFER_SIZE, ByteOrder.LITTLE_ENDIAN); + + private static final NonBlockingPool LITTLEST_END_BYTE_BUF_POOL = + makeBufferPool("littlestEndByteBufPool", SMALLEST_BUFFER_SIZE, ByteOrder.LITTLE_ENDIAN); + + private static final NonBlockingPool SHAPESHIFT_INTS_DECODED_VALUES_ARRAY_POOL = + makeIntArrayPool( + "shapeshiftIntsDecodedValuesArrayPool", + INT_ARRAY_SIZE, + INT_DECODED_ARRAY_POOL_MAX_CACHE + ); + + private static final NonBlockingPool SHAPESHIFT_INTS_ENCODED_VALUES_ARRAY_POOL = + makeIntArrayPool( + "shapeshiftIntsEncodedValuesArrayPool", + INT_ARRAY_SIZE + ENCODED_INTS_SHOULD_BE_ENOUGH, + INT_ENCODED_ARRAY_POOL_MAX_CACHE + ); + + private static final NonBlockingPool SHAPESHIFT_SMALLER_INTS_DECODED_VALUES_ARRAY_POOL = + makeIntArrayPool( + "shapeshiftSmallerIntsDecodedValuesArrayPool", + SMALLER_INT_ARRAY_SIZE, + INT_DECODED_ARRAY_POOL_MAX_CACHE + ); + + private static final NonBlockingPool SHAPESHIFT_SMALLER_INTS_ENCODED_VALUES_ARRAY_POOL = + makeIntArrayPool( + "shapeshiftSmallerIntsEncodedValuesArrayPool", + SMALLER_INT_ARRAY_SIZE + ENCODED_INTS_SHOULD_BE_ENOUGH, + INT_ENCODED_ARRAY_POOL_MAX_CACHE + ); + + private static final NonBlockingPool SHAPESHIFT_SMALLEST_INTS_DECODED_VALUES_ARRAY_POOL = + makeIntArrayPool( + "shapeshiftSmallestIntsDecodedValuesArrayPool", + SMALLEST_INT_ARRAY_SIZE, + INT_DECODED_ARRAY_POOL_MAX_CACHE + ); + + private static final NonBlockingPool SHAPESHIFT_SMALLEST_INTS_ENCODED_VALUES_ARRAY_POOL = + makeIntArrayPool( + "shapeshiftSmallestIntsEncodedValuesArrayPool", + SMALLEST_INT_ARRAY_SIZE + ENCODED_INTS_SHOULD_BE_ENOUGH, + INT_ENCODED_ARRAY_POOL_MAX_CACHE + ); + + + private static final NonBlockingPool SHAPESHIFT_FAST_PFOR_CODEC_POOL = + makeFastpforPool( + "shapeshiftFastPforCodecPool", + INT_ARRAY_SIZE + ); + public static ResourceHolder getByteBuf(ByteOrder order) { - if (order == ByteOrder.LITTLE_ENDIAN) { - return LITTLE_ENDIAN_BYTE_BUF_POOL.take(); + if (order.equals(ByteOrder.LITTLE_ENDIAN)) { + return LITTLE_END_BYTE_BUF_POOL.take(); + } + return BIG_END_BYTE_BUF_POOL.take(); + } + + private static ResourceHolder getSmallerByteBuf(ByteOrder order) + { + if (order.equals(ByteOrder.LITTLE_ENDIAN)) { + return LITTLER_END_BYTE_BUF_POOL.take(); + } + return LITTLE_BIG_END_BYTE_BUF_POOL.take(); + } + + private static ResourceHolder getSmallestByteBuf(ByteOrder order) + { + if (order.equals(ByteOrder.LITTLE_ENDIAN)) { + return LITTLEST_END_BYTE_BUF_POOL.take(); + } + return LITTLEST_BIG_END_BYTE_BUF_POOL.take(); + } + + /** + * Get pooled decoded values buffer for {@link ShapeShiftingColumnarInts} + * @param logBytesPerChunk + * @param order + * @return + */ + public static ResourceHolder getShapeshiftDecodedValuesBuffer(int logBytesPerChunk, ByteOrder order) + { + switch (logBytesPerChunk) { + case 14: + return getSmallestByteBuf(order); + case 15: + return getSmallerByteBuf(order); + case 16: + default: + return getByteBuf(order); + } + } + + + /** + * Get pooled decoded values array for {@link ShapeShiftingColumnarInts} + * @param logValuesPerChunk + * @return + */ + public static ResourceHolder getShapeshiftIntsDecodedValuesArray(int logValuesPerChunk) + { + switch (logValuesPerChunk) { + case 12: + return SHAPESHIFT_SMALLEST_INTS_DECODED_VALUES_ARRAY_POOL.take(); + case 13: + return SHAPESHIFT_SMALLER_INTS_DECODED_VALUES_ARRAY_POOL.take(); + case 14: + default: + return SHAPESHIFT_INTS_DECODED_VALUES_ARRAY_POOL.take(); + } + } + + /** + * Get pooled encoded values array for {@link ShapeShiftingColumnarInts} + * @param logValuesPerChunk + * @return + */ + public static ResourceHolder getShapeshiftIntsEncodedValuesArray(int logValuesPerChunk) + { + switch (logValuesPerChunk) { + case 12: + return SHAPESHIFT_SMALLEST_INTS_ENCODED_VALUES_ARRAY_POOL.take(); + case 13: + return SHAPESHIFT_SMALLER_INTS_ENCODED_VALUES_ARRAY_POOL.take(); + case 14: + default: + return SHAPESHIFT_INTS_ENCODED_VALUES_ARRAY_POOL.take(); + } + } + + public static NonBlockingPool getShapeshiftFastPforPool(int logValuesPerChunk) + { + switch (logValuesPerChunk) { + case 12: + case 13: + case 14: + default: + return SHAPESHIFT_FAST_PFOR_CODEC_POOL; + } + } + + public static NonBlockingPool getShapeshiftLemirePool(byte header, int logValuesPerChunk) + { + switch (header) { + case IntCodecs.FASTPFOR: + default: + return getShapeshiftFastPforPool(logValuesPerChunk); } - return BIG_ENDIAN_BYTE_BUF_POOL.take(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index 13101de94db0..299895c6108c 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -21,9 +21,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Sets; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.data.BitmapSerde; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CompressionFactory; @@ -46,6 +48,8 @@ public class IndexSpec public static final CompressionStrategy DEFAULT_METRIC_COMPRESSION = CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY; public static final CompressionStrategy DEFAULT_DIMENSION_COMPRESSION = CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY; public static final CompressionFactory.LongEncodingStrategy DEFAULT_LONG_ENCODING = CompressionFactory.DEFAULT_LONG_ENCODING_STRATEGY; + public static final ColumnEncodingStrategy DEFAULT_INT_ENCODING_STRATEGY = + new ColumnEncodingStrategy(EncodingStrategy.COMPRESSION, null, null); private static final Set METRIC_COMPRESSION = Sets.newHashSet( Arrays.asList(CompressionStrategy.values()) @@ -64,6 +68,7 @@ public class IndexSpec private final CompressionStrategy metricCompression; private final CompressionFactory.LongEncodingStrategy longEncoding; + private final ColumnEncodingStrategy intEncodingStrategy; @Nullable private final SegmentizerFactory segmentLoader; @@ -72,7 +77,7 @@ public class IndexSpec */ public IndexSpec() { - this(null, null, null, null, null); + this(null, null, null, null, null, null); } @VisibleForTesting @@ -83,16 +88,15 @@ public IndexSpec( @Nullable CompressionFactory.LongEncodingStrategy longEncoding ) { - this(bitmapSerdeFactory, dimensionCompression, metricCompression, longEncoding, null); + this(bitmapSerdeFactory, dimensionCompression, metricCompression, longEncoding, null, null); } /** * Creates an IndexSpec with the given storage format settings. * - * - * @param bitmapSerdeFactory type of bitmap to use (e.g. roaring or concise), null to use the default. - * Defaults to the bitmap type specified by the (deprecated) "druid.processing.bitmap.type" - * setting, or, if none was set, uses the default defined in {@link BitmapSerde} + * @param bitmapSerdeFactory type of bitmap to use (e.g. roaring or concise), null to use the default. + * Defaults to the bitmap type specified by the (deprecated) "druid.processing.bitmap.type" + * setting, or, if none was set, uses the default defined in {@link BitmapSerde} * * @param dimensionCompression compression format for dimension columns, null to use the default. * Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY} @@ -102,6 +106,8 @@ public IndexSpec( * * @param longEncoding encoding strategy for metric and dimension columns with type long, null to use the default. * Defaults to {@link CompressionFactory#DEFAULT_LONG_ENCODING_STRATEGY} + * + * @param intEncodingStrategy encoding strategy for integer columns */ @JsonCreator public IndexSpec( @@ -109,17 +115,27 @@ public IndexSpec( @JsonProperty("dimensionCompression") @Nullable CompressionStrategy dimensionCompression, @JsonProperty("metricCompression") @Nullable CompressionStrategy metricCompression, @JsonProperty("longEncoding") @Nullable CompressionFactory.LongEncodingStrategy longEncoding, - @JsonProperty("segmentLoader") @Nullable SegmentizerFactory segmentLoader + @JsonProperty("segmentLoader") @Nullable SegmentizerFactory segmentLoader, + @JsonProperty("intEncodingStrategy") @Nullable ColumnEncodingStrategy intEncodingStrategy ) { - Preconditions.checkArgument(dimensionCompression == null || DIMENSION_COMPRESSION.contains(dimensionCompression), - "Unknown compression type[%s]", dimensionCompression); + Preconditions.checkArgument( + dimensionCompression == null || DIMENSION_COMPRESSION.contains(dimensionCompression), + "Unknown compression type[%s]", + dimensionCompression + ); - Preconditions.checkArgument(metricCompression == null || METRIC_COMPRESSION.contains(metricCompression), - "Unknown compression type[%s]", metricCompression); + Preconditions.checkArgument( + metricCompression == null || METRIC_COMPRESSION.contains(metricCompression), + "Unknown compression type[%s]", + metricCompression + ); - Preconditions.checkArgument(longEncoding == null || LONG_ENCODING_NAMES.contains(longEncoding), - "Unknown long encoding type[%s]", longEncoding); + Preconditions.checkArgument( + longEncoding == null || LONG_ENCODING_NAMES.contains(longEncoding), + "Unknown long encoding type[%s]", + longEncoding + ); this.bitmapSerdeFactory = bitmapSerdeFactory != null ? bitmapSerdeFactory @@ -128,6 +144,7 @@ public IndexSpec( this.metricCompression = metricCompression == null ? DEFAULT_METRIC_COMPRESSION : metricCompression; this.longEncoding = longEncoding == null ? DEFAULT_LONG_ENCODING : longEncoding; this.segmentLoader = segmentLoader; + this.intEncodingStrategy = intEncodingStrategy == null ? DEFAULT_INT_ENCODING_STRATEGY : intEncodingStrategy; } @JsonProperty("bitmap") @@ -161,6 +178,12 @@ public SegmentizerFactory getSegmentLoader() return segmentLoader; } + @JsonProperty + public ColumnEncodingStrategy getIntEncodingStrategy() + { + return intEncodingStrategy; + } + @Override public boolean equals(Object o) { @@ -175,13 +198,21 @@ public boolean equals(Object o) dimensionCompression == indexSpec.dimensionCompression && metricCompression == indexSpec.metricCompression && longEncoding == indexSpec.longEncoding && - Objects.equals(segmentLoader, indexSpec.segmentLoader); + Objects.equals(segmentLoader, indexSpec.segmentLoader) && + intEncodingStrategy.equals(indexSpec.intEncodingStrategy); } @Override public int hashCode() { - return Objects.hash(bitmapSerdeFactory, dimensionCompression, metricCompression, longEncoding, segmentLoader); + return Objects.hash( + bitmapSerdeFactory, + dimensionCompression, + metricCompression, + longEncoding, + segmentLoader, + intEncodingStrategy + ); } @Override @@ -193,6 +224,193 @@ public String toString() ", metricCompression=" + metricCompression + ", longEncoding=" + longEncoding + ", segmentLoader=" + segmentLoader + + ", intEncodingStrategy=" + intEncodingStrategy + '}'; } + + /** + * Encapsulate column encoding strategy options + */ + public static class ColumnEncodingStrategy + { + private static final EncodingStrategy DEFAULT_ENCODING_STRATEGY = EncodingStrategy.COMPRESSION; + private static final ShapeShiftOptimizationTarget DEFAULT_OPTIMIZATION_TARGET = ShapeShiftOptimizationTarget.FASTBUTSMALLISH; + private static final ShapeShiftBlockSize DEFAULT_BLOCK_SIZE = ShapeShiftBlockSize.LARGE; + private static final Set ENCODING_STRATEGIES = Sets.newHashSet( + Arrays.asList(EncodingStrategy.values()) + ); + private static final Set OPTIMIZATION_TARGETS = Sets.newHashSet( + Arrays.asList(ShapeShiftOptimizationTarget.values()) + ); + private static final Set BLOCK_SIZES = Sets.newHashSet( + Arrays.asList(ShapeShiftBlockSize.values()) + ); + + private final EncodingStrategy strategy; + private final ShapeShiftOptimizationTarget optimizationTarget; + private final ShapeShiftBlockSize blockSize; + + @JsonCreator + public ColumnEncodingStrategy( + @JsonProperty("strategy") EncodingStrategy strategy, + @JsonProperty("optimizationTarget") ShapeShiftOptimizationTarget optimizationTarget, + @JsonProperty("blockSize") ShapeShiftBlockSize blockSize + ) + { + Preconditions.checkArgument(strategy == null || ENCODING_STRATEGIES.contains(strategy), + "Unknown encoding strategy[%s]", strategy + ); + Preconditions.checkArgument(optimizationTarget == null || OPTIMIZATION_TARGETS.contains(optimizationTarget), + "Unknown shapeshift optimization target[%s]", optimizationTarget + ); + Preconditions.checkArgument(blockSize == null || BLOCK_SIZES.contains(blockSize), + "Unknown shapeshift block size[%s]", blockSize + ); + this.strategy = strategy == null ? DEFAULT_ENCODING_STRATEGY : strategy; + this.optimizationTarget = optimizationTarget == null ? DEFAULT_OPTIMIZATION_TARGET : optimizationTarget; + this.blockSize = blockSize == null ? DEFAULT_BLOCK_SIZE : blockSize; + } + + @JsonProperty + public EncodingStrategy getStrategy() + { + return strategy; + } + + @JsonProperty + public ShapeShiftOptimizationTarget getOptimizationTarget() + { + return optimizationTarget; + } + + @JsonProperty + public ShapeShiftBlockSize getBlockSize() + { + return blockSize; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColumnEncodingStrategy that = (ColumnEncodingStrategy) o; + return strategy == that.strategy && + optimizationTarget == that.optimizationTarget && + blockSize == that.blockSize; + } + + @Override + public int hashCode() + { + + return Objects.hash(strategy, optimizationTarget, blockSize); + } + + @Override + public String toString() + { + return "ColumnEncodingStrategy{" + + "strategy=" + strategy + + ", optimizationTarget=" + optimizationTarget + + ", blockSize=" + blockSize + + '}'; + } + } + + + public enum EncodingStrategy + { + COMPRESSION, + SHAPESHIFT; + + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + public static EncodingStrategy fromString(String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } + } + + /** + * Log base 2 values per chunk in shapeshift encoding + */ + public enum ShapeShiftBlockSize + { + /** + * Shapeshift will encode blocks of 2^16 bytes. This puts the most memory pressure at indexing and query time in + * exchange for the potential to reduce encoded size. Approximate footprint is 64k off heap for decompression buffer + * and 129k on heap for value arrays + */ + LARGE(16), + /** + * Shapeshift will encode blocks of 2^15 bytes. Approximate footprint is 32k off heap for decompression buffer + * and 65k on heap for value arrays + */ + MIDDLE(15), + /** + * Shapeshift will encode blocks of 2^14 bytes. This approach is very conservative and uses less overall memory + * than {@link IndexSpec.EncodingStrategy#COMPRESSION} in exchange for increased encoding size overhead and + * potentially smaller gains in overall encoded size. Approximate footprint is 16k off heap for decompression buffer + * and 33k on heap for value arrays. + */ + SMALL(14); + + int logBlockSize; + + ShapeShiftBlockSize(int blockSize) + { + this.logBlockSize = blockSize; + } + + public byte getLogBlockSize() + { + return (byte) this.logBlockSize; + } + + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + public static ShapeShiftBlockSize fromString(String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } + } + + public enum ShapeShiftOptimizationTarget + { + SMALLER, + FASTBUTSMALLISH, + FASTER; + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + public static ShapeShiftOptimizationTarget fromString(String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java index abb4637dc7b8..9fbed617bfa3 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java @@ -54,10 +54,12 @@ import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ListIndexed; +import org.apache.druid.segment.data.ShapeShiftingColumnarIntsSerializer; import org.apache.druid.segment.data.SingleValueColumnarIntsSerializer; import org.apache.druid.segment.data.V3CompressedVSizeColumnarMultiIntsSerializer; import org.apache.druid.segment.data.VSizeColumnarIntsSerializer; import org.apache.druid.segment.data.VSizeColumnarMultiIntsSerializer; +import org.apache.druid.segment.data.codecs.ints.IntFormEncoder; import org.apache.druid.segment.serde.DictionaryEncodedColumnPartSerde; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; @@ -65,6 +67,7 @@ import javax.annotation.Nullable; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.ByteOrder; import java.nio.IntBuffer; import java.util.ArrayList; import java.util.Collections; @@ -219,6 +222,7 @@ private void writeDictionary(Iterable dictionaryValues) throws IOExcepti protected void setupEncodedValueWriter() throws IOException { final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); + final IndexSpec.ColumnEncodingStrategy intEncodingStrategy = indexSpec.getIntEncodingStrategy(); String filenameBase = StringUtils.format("%s.forward_dim", dimensionName); if (capabilities.hasMultipleValues().isTrue()) { @@ -235,16 +239,36 @@ protected void setupEncodedValueWriter() throws IOException new VSizeColumnarMultiIntsSerializer(dimensionName, segmentWriteOutMedium, cardinality); } } else { - if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { - encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( - dimensionName, - segmentWriteOutMedium, - filenameBase, - cardinality, - compressionStrategy - ); + if (intEncodingStrategy.getStrategy().equals(IndexSpec.EncodingStrategy.COMPRESSION)) { + if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { + encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create( + dimensionName, + segmentWriteOutMedium, + filenameBase, + cardinality, + compressionStrategy + ); + } else { + encodedValueSerializer = new VSizeColumnarIntsSerializer(segmentWriteOutMedium, cardinality); + } } else { - encodedValueSerializer = new VSizeColumnarIntsSerializer(segmentWriteOutMedium, cardinality); + final ByteOrder byteOrder = IndexIO.BYTE_ORDER; + // todo: allow specification of encoders on index spec? + IntFormEncoder[] intEncoders = ShapeShiftingColumnarIntsSerializer.getDefaultIntFormEncoders( + intEncodingStrategy.getBlockSize(), + compressionStrategy, + segmentWriteOutMedium.getCloser(), + byteOrder + ); + + encodedValueSerializer = + new ShapeShiftingColumnarIntsSerializer( + segmentWriteOutMedium, + intEncoders, + intEncodingStrategy.getOptimizationTarget(), + intEncodingStrategy.getBlockSize(), + byteOrder + ); } } encodedValueSerializer.open(); @@ -536,6 +560,7 @@ public ColumnDescriptor makeColumnDescriptor() boolean hasMultiValue = capabilities.hasMultipleValues().isTrue(); final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); + final IndexSpec.ColumnEncodingStrategy intEncodingStrategy = indexSpec.getIntEncodingStrategy(); final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); builder.setValueType(ValueType.STRING); @@ -546,7 +571,8 @@ public ColumnDescriptor makeColumnDescriptor() .withValue( encodedValueSerializer, hasMultiValue, - compressionStrategy != CompressionStrategy.UNCOMPRESSED + compressionStrategy != CompressionStrategy.UNCOMPRESSED, + intEncodingStrategy.getStrategy().equals(IndexSpec.EncodingStrategy.SHAPESHIFT) ) .withBitmapSerdeFactory(bitmapSerdeFactory) .withBitmapIndex(bitmapWriter) diff --git a/processing/src/main/java/org/apache/druid/segment/column/DoublesColumn.java b/processing/src/main/java/org/apache/druid/segment/column/DoublesColumn.java index 34ab5bb0394b..227a254b01a2 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/DoublesColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/DoublesColumn.java @@ -86,4 +86,9 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("column", column); } + + public ColumnarDoubles getColumn() + { + return column; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/FloatsColumn.java b/processing/src/main/java/org/apache/druid/segment/column/FloatsColumn.java index 6684e6a6b036..e09e2cbc3166 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/FloatsColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/FloatsColumn.java @@ -86,4 +86,9 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("column", column); } + + public ColumnarFloats getColumn() + { + return column; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ValueType.java b/processing/src/main/java/org/apache/druid/segment/column/ValueType.java index 4bdbcfb87fe0..029b2ca35ca4 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ValueType.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ValueType.java @@ -90,6 +90,13 @@ public SettableColumnValueSelector makeNewSettableColumnValueSelector() return new SettableLongColumnValueSelector(); } }, + INT { // todo: lame, this is for benchmark value generation + @Override + public SettableColumnValueSelector makeNewSettableColumnValueSelector() + { + return new SettableDimensionValueSelector(); + } + }, STRING { @Override public SettableColumnValueSelector makeNewSettableColumnValueSelector() diff --git a/processing/src/main/java/org/apache/druid/segment/data/ColumnarMultiIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/ColumnarMultiIntsSerializer.java index 6553a3c035c1..4d146a36e91e 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/ColumnarMultiIntsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/ColumnarMultiIntsSerializer.java @@ -24,7 +24,7 @@ /** * Serializer that produces {@link ColumnarMultiInts}. */ -public abstract class ColumnarMultiIntsSerializer implements ColumnarIntsSerializer +public interface ColumnarMultiIntsSerializer extends ColumnarIntsSerializer { - public abstract void addValues(IndexedInts ints) throws IOException; + void addValues(IndexedInts ints) throws IOException; } diff --git a/processing/src/main/java/org/apache/druid/segment/data/CompressedColumnarIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/CompressedColumnarIntsSerializer.java index a0442d95adfd..c32bb7b0b98a 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/CompressedColumnarIntsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/CompressedColumnarIntsSerializer.java @@ -33,7 +33,7 @@ /** * Streams array of integers out in the binary format described by {@link CompressedColumnarIntsSupplier} */ -public class CompressedColumnarIntsSerializer extends SingleValueColumnarIntsSerializer +public class CompressedColumnarIntsSerializer implements SingleValueColumnarIntsSerializer { private static final byte VERSION = CompressedColumnarIntsSupplier.VERSION; diff --git a/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java index 6060be2cc3c0..ac0753d942d4 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSerializer.java @@ -35,7 +35,7 @@ /** * Streams array of integers out in the binary format described by {@link CompressedVSizeColumnarIntsSupplier} */ -public class CompressedVSizeColumnarIntsSerializer extends SingleValueColumnarIntsSerializer +public class CompressedVSizeColumnarIntsSerializer implements SingleValueColumnarIntsSerializer { private static final byte VERSION = CompressedVSizeColumnarIntsSupplier.VERSION; diff --git a/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java b/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java index 84d8b8d7ad72..134cb6adbe34 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplier.java @@ -309,7 +309,7 @@ protected int _get(ByteBuffer buffer, boolean bigEngian, int bufferIndex) } } - private class CompressedVSizeColumnarInts implements ColumnarInts + protected class CompressedVSizeColumnarInts implements ColumnarInts { final Indexed> singleThreadedBuffers = baseBuffers.singleThreaded(); diff --git a/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java b/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java index 1d229b51c8ac..e085a9ebf1b5 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java +++ b/processing/src/main/java/org/apache/druid/segment/data/CompressionStrategy.java @@ -161,9 +161,23 @@ public static CompressionStrategy[] noNoneValues() public interface Decompressor { /** - * Implementations of this method are expected to call out.flip() after writing to the output buffer + * Decompress data starting from {@code in.position()} to {@code out}. Implementations of this method are + * expected to call {@code out.flip()} after writing to the output buffer. + * @param in input buffer + * @param numBytes number of bytes to read + * @param out destination buffer for decompressed data */ void decompress(ByteBuffer in, int numBytes, ByteBuffer out); + + /** + * Decompress data starting from offset specified by {@code inOffset} of {@code in} to {@code out}. Implementations + * of this method are expected to call {@code out.flip()} after writing to the output buffer. + * @param in input buffer + * @param inOffset input buffer start position + * @param numBytes number of bytes to read + * @param out destination buffer for decompressed data + */ + void decompress(ByteBuffer in, int inOffset, int numBytes, ByteBuffer out); } public abstract static class Compressor @@ -174,7 +188,7 @@ public abstract static class Compressor * * If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer. */ - ByteBuffer allocateInBuffer(int inputSize, Closer closer) + public ByteBuffer allocateInBuffer(int inputSize, Closer closer) { return ByteBuffer.allocate(inputSize); } @@ -187,7 +201,7 @@ ByteBuffer allocateInBuffer(int inputSize, Closer closer) * * If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer. */ - abstract ByteBuffer allocateOutBuffer(int inputSize, Closer closer); + public abstract ByteBuffer allocateOutBuffer(int inputSize, Closer closer); /** * Returns a ByteBuffer with compressed contents of in between it's position and limit. It may be the provided out @@ -205,7 +219,7 @@ public static class UncompressedCompressor extends Compressor private static final UncompressedCompressor DEFAULT_COMPRESSOR = new UncompressedCompressor(); @Override - ByteBuffer allocateOutBuffer(int inputSize, Closer closer) + public ByteBuffer allocateOutBuffer(int inputSize, Closer closer) { return ByteBuffer.allocate(inputSize); } @@ -230,6 +244,16 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) in.position(in.position() + numBytes); } + @Override + public void decompress(ByteBuffer in, int inOffset, int numBytes, ByteBuffer out) + { + final ByteBuffer view = in.asReadOnlyBuffer(); + view.position(inOffset); + final ByteBuffer copyBuffer = view.slice().duplicate(); + copyBuffer.limit(copyBuffer.position() + numBytes); + out.put(copyBuffer).flip(); + in.position(in.position() + numBytes); // todo: idk what to do here.. this method was introduced to not care about underlying buffer position + } } public static class LZFDecompressor implements Decompressor @@ -253,6 +277,25 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) } } + @Override + public void decompress(ByteBuffer in, int inOffset, int numBytes, ByteBuffer out) + { + final byte[] bytes = new byte[numBytes]; + for (int i = 0, pos = inOffset; i < numBytes; i++, pos++) { + bytes[i] = in.get(pos); + } + + try (final ResourceHolder outputBytesHolder = CompressedPools.getOutputBytes()) { + final byte[] outputBytes = outputBytesHolder.get(); + final int numDecompressedBytes = LZFDecoder.decode(bytes, outputBytes); + out.put(outputBytes, 0, numDecompressedBytes); + out.flip(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + } public static class LZFCompressor extends Compressor @@ -305,6 +348,19 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) out.limit(out.position() + numDecompressedBytes); } + @Override + public void decompress(ByteBuffer in, int inOffset, int numBytes, ByteBuffer out) + { + final int numDecompressedBytes = LZ4_SAFE.decompress( + in, + inOffset, + numBytes, + out, + out.position(), + out.remaining() + ); + out.limit(out.position() + numDecompressedBytes); + } } public static class LZ4Compressor extends Compressor @@ -317,7 +373,7 @@ public static class LZ4Compressor extends Compressor } @Override - ByteBuffer allocateInBuffer(int inputSize, Closer closer) + public ByteBuffer allocateInBuffer(int inputSize, Closer closer) { ByteBuffer inBuffer = ByteBuffer.allocateDirect(inputSize); closer.register(() -> ByteBufferUtils.free(inBuffer)); @@ -325,7 +381,7 @@ ByteBuffer allocateInBuffer(int inputSize, Closer closer) } @Override - ByteBuffer allocateOutBuffer(int inputSize, Closer closer) + public ByteBuffer allocateOutBuffer(int inputSize, Closer closer) { ByteBuffer outBuffer = ByteBuffer.allocateDirect(LZ4_HIGH.maxCompressedLength(inputSize)); closer.register(() -> ByteBufferUtils.free(outBuffer)); diff --git a/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingBlockColumnarInts.java b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingBlockColumnarInts.java new file mode 100644 index 000000000000..e5926d770571 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingBlockColumnarInts.java @@ -0,0 +1,63 @@ +/* + * 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.data; + +import it.unimi.dsi.fastutil.bytes.Byte2ObjectMap; +import org.apache.druid.segment.data.codecs.ConstantFormDecoder; +import org.apache.druid.segment.data.codecs.FormDecoder; + +import java.util.Arrays; + +/** + * Variant of {@link ShapeShiftingColumnarInts} that is optimized for eagerly decoding all column values, allowing + * {@link ShapeShiftingColumnarInts#get(int)} to be implemented directly as a masked array access. + * This optimization will be produced by {@link ShapeShiftingColumnarIntsSupplier} if + */ +public final class ShapeShiftingBlockColumnarInts extends ShapeShiftingColumnarInts +{ + public ShapeShiftingBlockColumnarInts( + ShapeShiftingColumnData sourceData, + Byte2ObjectMap> decoders + ) + { + super(sourceData, decoders); + } + + @Override + public int get(final int index) + { + final int desiredChunk = index >> logValuesPerChunk; + + if (desiredChunk != currentChunk) { + loadChunk(desiredChunk); + } + + return decodedValues[index & chunkIndexMask]; + } + + @Override + public void transform(FormDecoder nextForm) + { + nextForm.transform(this); + if (nextForm instanceof ConstantFormDecoder) { + Arrays.fill(getDecodedValues(), 0, currentChunkNumValues, getCurrentConstant()); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumn.java b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumn.java new file mode 100644 index 000000000000..78e842e6c2ea --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumn.java @@ -0,0 +1,338 @@ +/* + * 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.data; + +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import it.unimi.dsi.fastutil.bytes.Byte2ObjectMap; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.CompressedPools; +import org.apache.druid.segment.data.codecs.FormDecoder; +import sun.misc.Unsafe; +import sun.nio.ch.DirectBuffer; + +import java.io.Closeable; +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Base type for reading 'shape shifting' columns, which divide row values into chunks sized to a power of 2, each + * potentially encoded with a different algorithm which was chosen as optimal for size and speed for the given values + * at indexing time. This class generically provides common structure for loading and decoding chunks of values for + * all shape shifting column implementations, with the help of matching {@link FormDecoder}. Like + * some other column decoding strategies, shape shifting columns operate with a 'currentChunk' which is loaded when + * a 'get' operation for a row index is done, and remains until a row index on a different chunk is requested, so + * performs best of row selection is done in an ordered manner. + * + * Shape shifting columns are designed to place row retrieval functions within the column implementation for optimal + * performance with the jvm. Each chunk has a byte header that uniquely maps to a {@link FormDecoder}, andChunks are + * decoded by passing them column into {@link FormDecoder} which are tightly coupled to know how to mutate the + * implementation so row values can be retrieved. What this means is implementation specific, but for the sake of + * example, could be decoding all values of the chunk to a primitive array or setting offsets to read values directly + * from a {@link ByteBuffer}. See specific implementations for further details. + * + * @param type of {@link ShapeShiftingColumn} implementation to strongly associate {@link FormDecoder} + */ +public abstract class ShapeShiftingColumn implements Closeable +{ + public static Unsafe getTheUnsafe() + { + try { + Field theUnsafe = Unsafe.class.getDeclaredField("theUnsafe"); + theUnsafe.setAccessible(true); + return (Unsafe) theUnsafe.get(null); + } + catch (Exception e) { + throw new AssertionError(e); + } + } + + final ByteBuffer buffer; + final int numChunks; + final int numValues; + final byte logValuesPerChunk; + final int valuesPerChunk; + final int chunkIndexMask; + final ByteOrder byteOrder; + // shared chunk data areas for decoders that need space + ResourceHolder bufHolder; + private final Supplier decompressedDataBuffer; + + protected int currentChunk = -1; + protected ByteBuffer currentValueBuffer; + protected ByteBuffer currentMetadataBuffer; + protected long currentValuesAddress; + protected int currentValuesStartOffset; + protected int currentChunkStartOffset; + protected int currentChunkSize; + protected int currentChunkNumValues; + + protected final Byte2ObjectMap> decoders; + final ShapeShiftingColumnData columnData; + + public ShapeShiftingColumn(ShapeShiftingColumnData sourceData, Byte2ObjectMap> decoders) + { + this.buffer = sourceData.getBaseBuffer(); + this.numChunks = sourceData.getNumChunks(); + this.numValues = sourceData.getNumValues(); + this.logValuesPerChunk = sourceData.getLogValuesPerChunk(); + this.valuesPerChunk = 1 << logValuesPerChunk; + this.chunkIndexMask = valuesPerChunk - 1; + this.byteOrder = sourceData.getByteOrder(); + this.decoders = decoders; + this.columnData = sourceData; + + // todo: meh side effects... + this.decompressedDataBuffer = Suppliers.memoize(() -> { + this.bufHolder = CompressedPools.getShapeshiftDecodedValuesBuffer( + logValuesPerChunk + sourceData.getLogBytesPerValue(), + byteOrder + ); + return this.bufHolder.get(); + }); + } + + @Override + public void close() throws IOException + { + if (bufHolder != null) { + bufHolder.close(); + } + } + + /** + * This method loads and decodes a chunk of values, and should be called in column 'get' methods to change the current + * chunk + * + * @param desiredChunk + */ + final void loadChunk(int desiredChunk) + { + // todo: needed? + //CHECKSTYLE.OFF: Regexp +// Preconditions.checkArgument( +// desiredChunk < numChunks, +// "desiredChunk[%s] < numChunks[%s]", +// desiredChunk, +// numChunks +// ); + //CHECKSTYLE.ON: Regexp + + currentValueBuffer = buffer; + currentMetadataBuffer = buffer; + currentValuesAddress = -1; + currentValuesStartOffset = -1; + currentChunkStartOffset = -1; + currentChunk = -1; + currentChunkSize = -1; + + if (desiredChunk == numChunks - 1) { + currentChunkNumValues = (numValues - ((numChunks - 1) * valuesPerChunk)); + } else { + currentChunkNumValues = valuesPerChunk; + } + + final ShapeShiftingColumnData.ChunkPosition position = columnData.getChunkPosition(desiredChunk); + final int chunkStartByte = columnData.getValueChunksStartOffset() + position.getStartOffset(); + final int chunkEndByte = columnData.getValueChunksStartOffset() + position.getEndOffset(); + final byte chunkCodec = buffer.get(chunkStartByte); + + FormDecoder nextForm = getFormDecoder(chunkCodec); + + currentChunkStartOffset = chunkStartByte + 1; + currentValuesStartOffset = currentChunkStartOffset + nextForm.getMetadataSize(); + currentChunkSize = chunkEndByte - currentValuesStartOffset; + if (buffer.isDirect() && byteOrder.equals(ByteOrder.nativeOrder())) { + currentValuesAddress = (((DirectBuffer) buffer).address() + currentValuesStartOffset); + } + + transform(nextForm); + + currentChunk = desiredChunk; + } + + public void inspectRuntimeShape(final RuntimeShapeInspector inspector) + { + // todo: idk + inspector.visit("decompressedDataBuffer", decompressedDataBuffer); + } + + /** + * Transform this shapeshifting column to be able to read row values for the specified chunk + * + * @param nextForm decoder for the form of the next chunk of values + */ + public abstract void transform(FormDecoder nextForm); + + /** + * Get form decoder mapped to chunk header, used to transform this column to prepare for value reading + * + * @param header + * + * @return + */ + public FormDecoder getFormDecoder(byte header) + { + return decoders.get(header); + } + + /** + * Get underlying column buffer sliced from mapped smoosh + * + * @return + */ + public ByteBuffer getBuffer() + { + return this.buffer; + } + + /** + * Get shared bytebuffer for decompression + * + * @return + */ + public ByteBuffer getDecompressedDataBuffer() + { + return this.decompressedDataBuffer.get(); + } + + /** + * Get current {@link ByteBuffer} to read row values from + * + * @return + */ + public final ByteBuffer getCurrentValueBuffer() + { + return currentValueBuffer; + } + + /** + * Set bytebuffer to read row values from + * + * @param currentValueBuffer + */ + public void setCurrentValueBuffer(ByteBuffer currentValueBuffer) + { + this.currentValueBuffer = currentValueBuffer; + } + + /** + * Get 'unsafe' memory address of current value chunk direct buffer + * + * @return + */ + public final long getCurrentValuesAddress() + { + return currentValuesAddress; + } + + /** + * Set 'unsafe' memory address of current value chunk direct buffer + * + * @param currentValuesAddress + */ + public final void setCurrentValuesAddress(long currentValuesAddress) + { + this.currentValuesAddress = currentValuesAddress; + } + + /** + * Get buffer offset of base column buffer for values of current chunk. If chunk has it's own encoding metadata, this + * may be offset from the start of the chunk itself. + * + * @return + */ + public final int getCurrentValuesStartOffset() + { + return currentValuesStartOffset; + } + + /** + * Set buffer offset of base column buffer for current value chunk. If chunk has it's own encoding metadata, this + * may be offset from the start of the chunk itself + */ + public final void setCurrentValuesStartOffset(int currentValuesStartOffset) + { + this.currentValuesStartOffset = currentValuesStartOffset; + } + + /** + * Get buffer offset of base column buffer for start of current chunk. + * + * @return + */ + public int getCurrentChunkStartOffset() + { + return currentChunkStartOffset; + } + + /** + * Set buffer offset of base column buffer for current chunk. + * + * @param currentChunkStartOffset + */ + public void setCurrentChunkStartOffset(int currentChunkStartOffset) + { + this.currentChunkStartOffset = currentChunkStartOffset; + } + + /** + * Get size in bytes of current chunk + * + * @return + */ + public int getCurrentChunkSize() + { + return currentChunkSize; + } + + /** + * Set size in bytes of current chunk + * + * @param currentChunkSize + */ + public void setCurrentChunkSize(int currentChunkSize) + { + this.currentChunkSize = currentChunkSize; + } + + /** + * Get number of rows in current chunk + * + * @return + */ + public int getCurrentChunkNumValues() + { + return currentChunkNumValues; + } + + /** + * Set number of rows in current chunk + * + * @param currentChunkNumValues + */ + public void setCurrentChunkNumValues(int currentChunkNumValues) + { + this.currentChunkNumValues = currentChunkNumValues; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnData.java b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnData.java new file mode 100644 index 000000000000..77bbe2b897d3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnData.java @@ -0,0 +1,247 @@ +/* + * 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.data; + +import it.unimi.dsi.fastutil.bytes.Byte2IntArrayMap; +import it.unimi.dsi.fastutil.bytes.Byte2IntMap; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Materialized version of outer buffer contents of a {@link ShapeShiftingColumn}, extracting all header information + * as well as a sliced buffer, prepared for reading, allowing suppliers a tidy structure to instantiate + * {@link ShapeShiftingColumn} objects. + * + * layout: + * | version (byte) | headerSize (int) | numValues (int) | numChunks (int) | logValuesPerChunk (byte) | offsetsSize (int) | compositionSize (int) | composition | offsets | values | + */ +public class ShapeShiftingColumnData +{ + private final int headerSize; + private final int numValues; + private final int numChunks; + private final byte logValuesPerChunk; + private final byte logBytesPerValue; + private final int compositionOffset; + private final int compositionSize; + private final int offsetsOffset; + private final int offsetsSize; + private final Byte2IntArrayMap composition; + private final ChunkPosition[] chunkPositions; + private final ByteBuffer baseBuffer; + private final ByteOrder byteOrder; + + public ShapeShiftingColumnData(ByteBuffer buffer, byte logBytesPerValue, ByteOrder byteOrder) + { + this(buffer, logBytesPerValue, byteOrder, false); + } + + public ShapeShiftingColumnData( + ByteBuffer buffer, + byte logBytesPerValue, + ByteOrder byteOrder, + boolean moveSourceBufferPosition + ) + { + ByteBuffer ourBuffer = buffer.slice().order(byteOrder); + int position = 0; + this.byteOrder = byteOrder; + this.logBytesPerValue = logBytesPerValue; + this.headerSize = ourBuffer.getInt(++position); + this.numValues = ourBuffer.getInt(position += Integer.BYTES); + this.numChunks = ourBuffer.getInt(position += Integer.BYTES); + this.logValuesPerChunk = ourBuffer.get(position += Integer.BYTES); + this.compositionOffset = ourBuffer.getInt(++position); + this.compositionSize = ourBuffer.getInt(position += Integer.BYTES); + this.offsetsOffset = ourBuffer.getInt(position += Integer.BYTES); + this.offsetsSize = ourBuffer.getInt(position += Integer.BYTES); + + this.composition = new Byte2IntArrayMap(); + // 5 bytes per composition entry + for (int i = 0; i < compositionSize; i += 5) { + byte header = ourBuffer.get(compositionOffset + i); + int count = ourBuffer.getInt(compositionOffset + i + 1); + composition.put(header, count); + } + + this.chunkPositions = new ChunkPosition[numChunks]; + int prevOffset = 0; + for (int chunk = 0, chunkOffset = 0; chunk < numChunks; chunk++, chunkOffset += Integer.BYTES) { + int chunkStart = ourBuffer.getInt(offsetsOffset + chunkOffset); + int chunkEnd = ourBuffer.getInt(offsetsOffset + chunkOffset + Integer.BYTES); + chunkPositions[chunk] = new ChunkPosition(chunkStart, chunkEnd); + prevOffset = chunkEnd; + } + + ourBuffer.limit(getValueChunksStartOffset() + prevOffset); + + if (moveSourceBufferPosition) { + buffer.position(buffer.position() + ourBuffer.remaining()); + } + + this.baseBuffer = ourBuffer.slice().order(byteOrder); + } + + /** + * Total 'header' size, to future proof by allowing us to always be able to find offsets and values sections offsets, + * but stuffing any additional data into the header. + * + * @return + */ + public int getHeaderSize() + { + return headerSize; + } + + /** + * Total number of rows in this column + * + * @return + */ + public int getNumValues() + { + return numValues; + } + + /** + * Number of 'chunks' of values this column is divided into + * + * @return + */ + public int getNumChunks() + { + return numChunks; + } + + /** + * log base 2 max number of values per chunk + * + * @return + */ + public byte getLogValuesPerChunk() + { + return logValuesPerChunk; + } + + /** + * log base 2 number of bytes per value + * + * @return + */ + public byte getLogBytesPerValue() + { + return logBytesPerValue; + } + + /** + * Size in bytes of chunk offset data + * + * @return + */ + public int getOffsetsSize() + { + return offsetsSize; + } + + /** + * Size in bytes of composition data + * + * @return + */ + public int getCompositionSize() + { + return compositionSize; + } + + /** + * Get composition of codecs used in column and their counts, allowing column suppliers to optimize at query time. + * Note that 'compressed' blocks are counted twice, once as compression and once as inner codec, so the total + * count here may not match {@link ShapeShiftingColumnData#numChunks} + * + * @return + */ + public Byte2IntMap getComposition() + { + return composition; + } + + /** + * get start and end offset of chunk in {@link ShapeShiftingColumnData#baseBuffer} + * @param chunk index of chunk + * @return + */ + public ChunkPosition getChunkPosition(int chunk) + { + return chunkPositions[chunk]; + } + + /** + * Start offset of {@link ShapeShiftingColumnData#baseBuffer} for the 'chunk values' section + * + * @return + */ + public int getValueChunksStartOffset() + { + return headerSize; + } + + /** + * {@link ByteBuffer} View of column data, sliced from underlying mapped segment smoosh buffer. + * + * @return + */ + public ByteBuffer getBaseBuffer() + { + return baseBuffer; + } + + /** + * Column byte order + * + * @return + */ + public ByteOrder getByteOrder() + { + return byteOrder; + } + + public static class ChunkPosition + { + private final int startOffset; + private final int endOffset; + + public ChunkPosition(int startOffset, int endOffset) + { + this.startOffset = startOffset; + this.endOffset = endOffset; + } + + public int getStartOffset() + { + return startOffset; + } + + public int getEndOffset() + { + return endOffset; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnSerializer.java new file mode 100644 index 000000000000..0ad36b01409b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnSerializer.java @@ -0,0 +1,280 @@ +/* + * 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.data; + +import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; +import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.codecs.CompressedFormEncoder; +import org.apache.druid.segment.data.codecs.FormEncoder; +import org.apache.druid.segment.data.codecs.FormMetrics; +import org.apache.druid.segment.serde.Serializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; +import java.util.Map; + +/** + * Base serializer for {@link ShapeShiftingColumn} implementations, providing most common functionality such as headers, + * value-chunking, encoder selection, and writing out values. + * + * Encoding Selection: + * The intention of this base structure is that implementors of this class will analyze incoming values and aggregate + * facts about the data which matching {@link FormEncoder} implementations might find interesting, while storing raw, + * unencoded values in {@link ShapeShiftingColumnSerializer#currentChunk}. When the threshold of + * {@link ShapeShiftingColumnSerializer#valuesPerChunk} is reached, {@link ShapeShiftingColumnSerializer} will attempt + * to find the "best" encoding by first computing the encoded size with + * {@link FormEncoder#getEncodedSize} and then applying a modifier to scale this value in order to influence behavior + * when sizes are relatively close according to the chosen {@link IndexSpec.ShapeShiftOptimizationTarget}. This + * effectively sways the decision towards using encodings with faster decoding speed or smaller encoded size as + * appropriate. Note that very often the best encoding is unambiguous and these settings don't matter, the nuanced + * differences of behavior of {@link IndexSpec.ShapeShiftOptimizationTarget} mainly come into play when things are + * close. + * + * Implementors need only supply an initialize method to allocate storage for {@code }, an add value method to + * populate {@code }, a reset method to prepare {@code } for the next chunk after a flush, and + * of course, matching {@link FormEncoder} implementations to perform actual value encoding. Generic compression is + * available to {@link FormEncoder} implementations by implementing + * {@link org.apache.druid.segment.data.codecs.CompressibleFormEncoder} and wrapping in a + * {@link CompressedFormEncoder} in the codec list passed to the serializer. + * + * layout: + * | version (byte) | headerSize (int) | numValues (int) | numChunks (int) | logValuesPerChunk (byte) | compositionOffset (int) | compositionSize (int) | offsetsOffset (int) | offsetsSize (int) | composition | offsets | values | + * + * @param + * @param + */ +public abstract class ShapeShiftingColumnSerializer implements Serializer +{ + /** + * | version (byte) | headerSize (int) | numValues (int) | numChunks (int) | logValuesPerChunk (byte) | compositionOffset (int) | compositionSize (int) | offsetsOffset (int) | offsetsSize (int) | + */ + private static final int BASE_HEADER_BYTES = 1 + (3 * Integer.BYTES) + 1 + (4 * Integer.BYTES); + + private static Logger log = new Logger(ShapeShiftingColumnSerializer.class); + + protected final SegmentWriteOutMedium segmentWriteOutMedium; + protected final FormEncoder[] codecs; + protected final byte version; + protected final byte logValuesPerChunk; + protected final int valuesPerChunk; + protected final ByteBuffer intToBytesHelperBuffer; + protected final Object2IntMap composition; + protected final IndexSpec.ShapeShiftOptimizationTarget optimizationTarget; + protected WriteOutBytes offsetsOut; + protected WriteOutBytes valuesOut; + protected boolean wroteFinalOffset = false; + protected TChunkMetrics chunkMetrics; + protected TChunk currentChunk; + protected int currentChunkPos = 0; + protected int numChunks = 0; + protected int numValues = 0; + + public ShapeShiftingColumnSerializer( + final SegmentWriteOutMedium segmentWriteOutMedium, + final FormEncoder[] codecs, + final IndexSpec.ShapeShiftOptimizationTarget optimizationTarget, + final IndexSpec.ShapeShiftBlockSize blockSize, + final int logBytesPerValue, + final byte version, + @Nullable final ByteOrder overrideByteOrder, + @Nullable final Byte overrideLogValuesPerChunk + ) + { + Preconditions.checkArgument(codecs.length > 0, "must have at least one encoder"); + this.segmentWriteOutMedium = Preconditions.checkNotNull(segmentWriteOutMedium, "segmentWriteOutMedium"); + this.version = version; + this.logValuesPerChunk = overrideLogValuesPerChunk != null + ? overrideLogValuesPerChunk + : (byte) (blockSize.getLogBlockSize() - logBytesPerValue); + this.valuesPerChunk = 1 << logValuesPerChunk; + this.codecs = codecs; + this.optimizationTarget = optimizationTarget; + ByteOrder byteOrder = overrideByteOrder == null ? ByteOrder.nativeOrder() : overrideByteOrder; + this.intToBytesHelperBuffer = ByteBuffer.allocate(Integer.BYTES).order(byteOrder); + this.composition = new Object2IntArrayMap<>(); + } + + public void open() throws IOException + { + offsetsOut = segmentWriteOutMedium.makeWriteOutBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); + initializeChunk(); + resetChunkCollector(); + } + + /** + * Initialize/allocate {@link ShapeShiftingColumnSerializer#currentChunk} to hold unencoded chunk values until + * {@link ShapeShiftingColumnSerializer#flushCurrentChunk()} is performed. + */ + public abstract void initializeChunk(); + + /** + * Reset {@link ShapeShiftingColumnSerializer#chunkMetrics} to prepare for analyzing the next incoming chunk of data + * after performing {@link ShapeShiftingColumnSerializer#flushCurrentChunk()} + */ + public abstract void resetChunkCollector(); + + @Override + public long getSerializedSize() throws IOException + { + if (currentChunkPos > 0) { + flushCurrentChunk(); + } + + writeFinalOffset(); + + return getHeaderSize() + valuesOut.size(); + } + + @Override + public void writeTo( + final WritableByteChannel channel, + final FileSmoosher smoosher + ) throws IOException + { + if (currentChunkPos > 0) { + flushCurrentChunk(); + } + + writeShapeShiftHeader(channel); + valuesOut.writeTo(channel); + } + + + protected ByteBuffer toBytes(final int n) + { + intToBytesHelperBuffer.putInt(0, n); + intToBytesHelperBuffer.rewind(); + return intToBytesHelperBuffer; + } + + /** + * Encode values of {@link ShapeShiftingColumnSerializer#currentChunk} with the 'best' available {@link FormEncoder} + * given the information collected in {@link ShapeShiftingColumnSerializer#chunkMetrics}. The best is chosen by + * computing the smallest 'modified' size, where {@link FormEncoder#getModifiedEncodedSize} is tuned based + * on decoding speed for each encoding in relation to all other available encodings. + * + * @throws IOException + */ + protected void flushCurrentChunk() throws IOException + { + Preconditions.checkState(!wroteFinalOffset, "!wroteFinalOffset"); + Preconditions.checkState(currentChunkPos > 0, "currentChunkPos > 0"); + Preconditions.checkState(offsetsOut.isOpen(), "offsetsOut.isOpen"); + Preconditions.checkState(valuesOut.isOpen(), "valuesOut.isOpen"); + + offsetsOut.write(toBytes(Ints.checkedCast(valuesOut.size()))); + + int bestSize = Integer.MAX_VALUE; + FormEncoder bestCodec = null; + if (codecs.length > 1) { + for (FormEncoder codec : codecs) { + double modifiedSize = codec.getModifiedEncodedSize(currentChunk, currentChunkPos, chunkMetrics); + if (modifiedSize < bestSize) { + bestCodec = codec; + bestSize = (int) modifiedSize; + } + } + } else { + bestCodec = codecs[0]; + } + + if (bestCodec == null) { + throw new RuntimeException("WTF? Unable to select an encoder."); + } + + if (!composition.containsKey(bestCodec)) { + composition.put(bestCodec, 0); + } + composition.computeIfPresent(bestCodec, (k, v) -> v + 1); + if (bestCodec instanceof CompressedFormEncoder) { + FormEncoder inner = ((CompressedFormEncoder) bestCodec).getInnerEncoder(); + if (!composition.containsKey(inner)) { + composition.put(inner, 0); + } + composition.computeIfPresent(inner, (k, v) -> v + 1); + } + valuesOut.write(new byte[]{bestCodec.getHeader()}); + bestCodec.encode(valuesOut, currentChunk, currentChunkPos, chunkMetrics); + + numChunks++; + resetChunk(); + } + + private void resetChunk() + { + currentChunkPos = 0; + resetChunkCollector(); + } + + + private int getHeaderSize() throws IOException + { + return BASE_HEADER_BYTES + (composition.size() * 5) + Ints.checkedCast(offsetsOut.size()); + } + + + private void writeFinalOffset() throws IOException + { + if (!wroteFinalOffset) { + offsetsOut.write(toBytes(Ints.checkedCast(valuesOut.size()))); + wroteFinalOffset = true; + } + } + + private void writeShapeShiftHeader(WritableByteChannel channel) throws IOException + { + writeFinalOffset(); + + int compositionSizeBytes = composition.entrySet().size() * 5; + int offsetsSizeBytes = Ints.checkedCast(offsetsOut.size()); + int headerSizeBytes = BASE_HEADER_BYTES + compositionSizeBytes + offsetsSizeBytes; + + channel.write(ByteBuffer.wrap(new byte[]{version})); + channel.write(toBytes(headerSizeBytes)); + channel.write(toBytes(numValues)); + channel.write(toBytes(numChunks)); + channel.write(ByteBuffer.wrap(new byte[]{logValuesPerChunk})); + channel.write(toBytes(BASE_HEADER_BYTES)); + channel.write(toBytes(compositionSizeBytes)); + channel.write(toBytes(BASE_HEADER_BYTES + compositionSizeBytes)); + channel.write(toBytes(offsetsSizeBytes)); + + // write composition map + for (Map.Entry enc : composition.entrySet()) { + channel.write(ByteBuffer.wrap(new byte[]{enc.getKey().getHeader()})); + channel.write(toBytes(enc.getValue())); + log.info(enc.getKey().getName() + ": " + enc.getValue()); + } + + // write offsets + offsetsOut.writeTo(channel); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarInts.java b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarInts.java new file mode 100644 index 000000000000..11b5625c722d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarInts.java @@ -0,0 +1,267 @@ +/* + * 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.data; + +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import it.unimi.dsi.fastutil.bytes.Byte2ObjectMap; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.CompressedPools; +import org.apache.druid.segment.data.codecs.ArrayFormDecoder; +import org.apache.druid.segment.data.codecs.CompressedFormDecoder; +import org.apache.druid.segment.data.codecs.FormDecoder; +import org.apache.druid.segment.data.codecs.ints.BytePackedIntFormDecoder; +import sun.misc.Unsafe; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class ShapeShiftingColumnarInts extends ShapeShiftingColumn implements ColumnarInts +{ + public static final byte VERSION = 0x4; // todo: idk.. + + protected static final Unsafe unsafe = getTheUnsafe(); + + protected final GetIntBuffer getInt24; + protected final GetIntUnsafe getInt24Unsafe; + ResourceHolder decodedValuesHolder; + + private final Supplier decodedValuesSupplier; + + protected int[] tmp; + protected int[] decodedValues; + protected DecodeIndex currentForm; + protected int currentBytesPerValue = 4; + protected int currentConstant = 0; + + public ShapeShiftingColumnarInts( + ShapeShiftingColumnData sourceData, + Byte2ObjectMap> decoders + ) + { + super(sourceData, decoders); + this.decodedValuesSupplier = Suppliers.memoize(() -> { + decodedValuesHolder = CompressedPools.getShapeshiftIntsDecodedValuesArray(logValuesPerChunk); + return decodedValuesHolder.get(); + }); + + getInt24 = byteOrder.equals(ByteOrder.LITTLE_ENDIAN) + ? (_buffer, pos) -> _buffer.getInt(pos) & BytePackedIntFormDecoder.LITTLE_ENDIAN_INT_24_MASK + : (_buffer, pos) -> _buffer.getInt(pos) >>> BytePackedIntFormDecoder.BIG_ENDIAN_INT_24_SHIFT; + getInt24Unsafe = byteOrder.equals(ByteOrder.LITTLE_ENDIAN) + ? (pos) -> unsafe.getInt(pos) & BytePackedIntFormDecoder.LITTLE_ENDIAN_INT_24_MASK + : (pos) -> unsafe.getInt(pos) >>> BytePackedIntFormDecoder.BIG_ENDIAN_INT_24_SHIFT; + } + + @Override + public int size() + { + return numValues; + } + + @Override + public void inspectRuntimeShape(final RuntimeShapeInspector inspector) + { + // todo: idk + super.inspectRuntimeShape(inspector); + inspector.visit("decodedValues", decodedValuesSupplier); + } + + @Override + public void close() throws IOException + { + super.close(); + if (decodedValuesHolder != null) { + decodedValuesHolder.close(); + } + } + + @Override + public int get(final int index) + { + final int desiredChunk = index >> logValuesPerChunk; + + if (desiredChunk != currentChunk) { + loadChunk(desiredChunk); + } + + return currentForm.decode(index & chunkIndexMask); + } + + /** + * integer array sized to number of values, to allow {@link FormDecoder} a place for fully + * decoded values upon transformation + * + * @return + */ + public final int[] getDecodedValues() + { + return decodedValues = decodedValuesSupplier.get(); + } + + /** + * current 'constant' value, for constant chunk transformations + * + * @return + */ + public final int getCurrentConstant() + { + return currentConstant; + } + + /** + * Allows {@link FormDecoder} to set current 'constant' value during a transformation. + * + * @param currentConstant + */ + public final void setCurrentConstant(int currentConstant) + { + this.currentConstant = currentConstant; + } + + /** + * Get current number of bytes used per value for random access transformations. + * + * @return + */ + public int getCurrentBytesPerValue() + { + return currentBytesPerValue; + } + + /** + * Allows {@link FormDecoder} to set current number of bytes for value, for random access transformations + * + * @param currentBytesPerValue + */ + public void setCurrentBytesPerValue(int currentBytesPerValue) + { + this.currentBytesPerValue = currentBytesPerValue; + } + + /** + * Transform {@link ShapeShiftingColumnarInts} to the form of the requested chunk, which may either be eagerly + * decoded entirely to {@link ShapeShiftingColumnarInts#decodedValuesSupplier} with values retrieved by + * {@link ShapeShiftingColumnarInts#decodeBlockForm(int)}, or randomly accessible, which may set + * {@link ShapeShiftingColumnarInts#currentValuesAddress}, {@link ShapeShiftingColumnarInts#currentValuesStartOffset}, + * {@link ShapeShiftingColumnarInts#currentBytesPerValue}, {@link ShapeShiftingColumnarInts#currentConstant} and be + * decoded by {@link ShapeShiftingColumnarInts#decodeBufferForm(int)}. + * + * @param nextForm + */ + @Override + public void transform(FormDecoder nextForm) + { + currentBytesPerValue = 4; + currentConstant = 0; + + nextForm.transform(this); + if (nextForm instanceof ArrayFormDecoder) { + currentForm = this::decodeBlockForm; + } else if (!(nextForm instanceof CompressedFormDecoder)) { + if (getCurrentValueBuffer().isDirect() && byteOrder.equals(ByteOrder.nativeOrder())) { + currentForm = this::decodeUnsafeForm; + } else { + currentForm = this::decodeBufferForm; + } + } + } + + /** + * get value at index produced {@link FormDecoder} transformation + * + * @param index masked index into the chunk array (index & {@link ShapeShiftingColumnarInts#chunkIndexMask}) + * + * @return decoded row value at index + */ + private int decodeBlockForm(int index) + { + return decodedValues[index]; + } + + /** + * get value (unsafe) at index produced by {@link FormDecoder} transformation + * + * @param index masked index into the chunk array (index & {@link ShapeShiftingColumnarInts#chunkIndexMask}) + * + * @return decoded row value at index + */ + private int decodeUnsafeForm(int index) + { + final long pos = currentValuesAddress + (index * currentBytesPerValue); + switch (currentBytesPerValue) { + case 1: + return unsafe.getByte(pos) & 0xFF; + case 2: + return unsafe.getShort(pos) & 0xFFFF; + case 3: + return getInt24Unsafe.getInt(pos); + case 4: + return unsafe.getInt(pos); + default: + return currentConstant; + } + } + + /** + * get value at index produced by {@link FormDecoder} transformation + * + * @param index masked index into the chunk array (index & {@link ShapeShiftingColumnarInts#chunkIndexMask}) + * + * @return decoded row value at index + */ + private int decodeBufferForm(int index) + { + final int pos = getCurrentValuesStartOffset() + (index * currentBytesPerValue); + final ByteBuffer buffer = getCurrentValueBuffer(); + switch (currentBytesPerValue) { + case 1: + return buffer.get(pos) & 0xFF; + case 2: + return buffer.getShort(pos) & 0xFFFF; + case 3: + return getInt24.getInt(buffer, pos); + case 4: + return buffer.getInt(pos); + default: + return currentConstant; + } + } + + @FunctionalInterface + public interface DecodeIndex + { + int decode(int index); + } + + @FunctionalInterface + protected interface GetIntUnsafe + { + int getInt(long index); + } + + @FunctionalInterface + protected interface GetIntBuffer + { + int getInt(ByteBuffer buffer, int index); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerializer.java new file mode 100644 index 000000000000..17642c876b4a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerializer.java @@ -0,0 +1,198 @@ +/* + * 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.data; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.CompressedPools; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.codecs.ints.BytePackedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.CompressedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.CompressibleIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.ConstantIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.IntCodecs; +import org.apache.druid.segment.data.codecs.ints.IntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.IntFormMetrics; +import org.apache.druid.segment.data.codecs.ints.LemireIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.RunLengthBytePackedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.UnencodedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.ZeroIntFormEncoder; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; + +/** + * {@link ShapeShiftingColumnSerializer} implementation for {@link ShapeShiftingColumnarInts}, using + * {@link IntFormEncoder} to encode values and {@link IntFormMetrics} to analyze them and assist with decisions of how + * the value chunks will be encoded when 'flushed' to the {@link SegmentWriteOutMedium}. + */ +public class ShapeShiftingColumnarIntsSerializer + extends ShapeShiftingColumnSerializer + implements SingleValueColumnarIntsSerializer +{ + private ResourceHolder unencodedValuesHolder; + + private final boolean enableEncoderOptOut; + + public static IntFormEncoder[] getDefaultIntFormEncoders( + IndexSpec.ShapeShiftBlockSize blockSize, + CompressionStrategy compressionStrategy, + Closer closer, + ByteOrder byteOrder + ) + { + byte intBlockSize = (byte) (blockSize.getLogBlockSize() - 2); + + ByteBuffer uncompressedDataBuffer = + compressionStrategy.getCompressor() + .allocateInBuffer(8 + ((1 << intBlockSize) * Integer.BYTES), closer) + .order(byteOrder); + ByteBuffer compressedDataBuffer = + compressionStrategy.getCompressor() + .allocateOutBuffer(((1 << intBlockSize) * Integer.BYTES) + 1024, closer); + + final CompressibleIntFormEncoder rle = new RunLengthBytePackedIntFormEncoder( + intBlockSize, + byteOrder + ); + final CompressibleIntFormEncoder bytepack = new BytePackedIntFormEncoder(intBlockSize, byteOrder); + + final IntFormEncoder[] defaultCodecs = new IntFormEncoder[]{ + new ZeroIntFormEncoder(intBlockSize, byteOrder), + new ConstantIntFormEncoder(intBlockSize, byteOrder), + new UnencodedIntFormEncoder(intBlockSize, byteOrder), + rle, + bytepack, + new CompressedIntFormEncoder( + intBlockSize, + byteOrder, + compressionStrategy, + rle, + uncompressedDataBuffer, + compressedDataBuffer + ), + new CompressedIntFormEncoder( + intBlockSize, + byteOrder, + compressionStrategy, + bytepack, + uncompressedDataBuffer, + compressedDataBuffer + ), + new LemireIntFormEncoder(intBlockSize, IntCodecs.FASTPFOR, "fastpfor", byteOrder) + }; + + return defaultCodecs; + } + + public ShapeShiftingColumnarIntsSerializer( + final SegmentWriteOutMedium segmentWriteOutMedium, + final IntFormEncoder[] codecs, + final IndexSpec.ShapeShiftOptimizationTarget optimizationTarget, + final IndexSpec.ShapeShiftBlockSize blockSize, + @Nullable final ByteOrder overrideByteOrder + ) + { + this( + segmentWriteOutMedium, + codecs, + optimizationTarget, + blockSize, + overrideByteOrder, + null + ); + } + + @VisibleForTesting + public ShapeShiftingColumnarIntsSerializer( + final SegmentWriteOutMedium segmentWriteOutMedium, + final IntFormEncoder[] codecs, + final IndexSpec.ShapeShiftOptimizationTarget optimizationTarget, + final IndexSpec.ShapeShiftBlockSize blockSize, + @Nullable final ByteOrder overrideByteOrder, + @Nullable final Byte overrideLogValuesPerChunk + ) + { + super( + segmentWriteOutMedium, + codecs, + optimizationTarget, + blockSize, + 2, + ShapeShiftingColumnarInts.VERSION, + overrideByteOrder, + overrideLogValuesPerChunk + ); + + Closer closer = segmentWriteOutMedium.getCloser(); + if (closer != null) { + closer.register(() -> { + if (unencodedValuesHolder != null) { + unencodedValuesHolder.close(); + } + }); + } + + // enable optimization of encoders such as rle if there are additional non-zero and non-constant encoders. + this.enableEncoderOptOut = + Arrays.stream(codecs) + .filter(e -> !(e instanceof ZeroIntFormEncoder) && !(e instanceof ConstantIntFormEncoder)) + .count() > 1; + } + + @Override + public void initializeChunk() + { + unencodedValuesHolder = CompressedPools.getShapeshiftIntsDecodedValuesArray(logValuesPerChunk); + currentChunk = unencodedValuesHolder.get(); + } + + @Override + public void resetChunkCollector() + { + chunkMetrics = new IntFormMetrics(optimizationTarget, this.enableEncoderOptOut); + } + + /** + * Adds a value to the current chunk of ints, stored in an array, analyzing values with {@link IntFormMetrics}, and + * flushing to the {@link SegmentWriteOutMedium} if the current chunk is full. + * + * @param val + * + * @throws IOException + */ + @Override + public void addValue(int val) throws IOException + { + if (currentChunkPos == valuesPerChunk) { + flushCurrentChunk(); + } + + chunkMetrics.processNextRow(val); + + currentChunk[currentChunkPos++] = val; + numValues++; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSupplier.java b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSupplier.java new file mode 100644 index 000000000000..3b0843565e84 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSupplier.java @@ -0,0 +1,201 @@ +/* + * 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.data; + +import com.google.common.annotations.VisibleForTesting; +import it.unimi.dsi.fastutil.bytes.Byte2IntMap; +import it.unimi.dsi.fastutil.bytes.Byte2ObjectMap; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.segment.data.codecs.ConstantFormDecoder; +import org.apache.druid.segment.data.codecs.DirectFormDecoder; +import org.apache.druid.segment.data.codecs.FormDecoder; +import org.apache.druid.segment.data.codecs.ints.IntCodecs; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +/** + * Reads mapped buffer contents into {@link ShapeShiftingColumnData} to supply {@link ShapeShiftingColumnarInts} + */ +public class ShapeShiftingColumnarIntsSupplier implements WritableSupplier +{ + private static Logger log = new Logger(ShapeShiftingColumnarIntsSupplier.class); + + private final ShapeShiftingColumnData columnData; + // null = default, false = force ShapeShiftingColumnarInts, true = force ShapeShiftingBlockColumnarInts + private final ShapeShiftingColumnarIntsDecodeOptimization overrideOptimization; + + private ShapeShiftingColumnarIntsSupplier( + ShapeShiftingColumnData columnData, + @Nullable ShapeShiftingColumnarIntsDecodeOptimization overrideOptimization + ) + { + this.columnData = columnData; + this.overrideOptimization = overrideOptimization; + } + + /** + * Create a new instance from a {@link ByteBuffer} with position set to the start of a + * {@link ShapeShiftingColumnarInts} + * + * @param buffer + * @param byteOrder + * + * @return + */ + public static ShapeShiftingColumnarIntsSupplier fromByteBuffer( + final ByteBuffer buffer, + final ByteOrder byteOrder + ) + { + ShapeShiftingColumnData columnData = + new ShapeShiftingColumnData(buffer, (byte) 2, byteOrder, true); + + return new ShapeShiftingColumnarIntsSupplier(columnData, null); + } + + /** + * Create a new instance from a {@link ByteBuffer} with position set to the start of a + * {@link ShapeShiftingColumnarInts} + * + * @param buffer + * @param byteOrder + * + * @return + */ + @VisibleForTesting + public static ShapeShiftingColumnarIntsSupplier fromByteBuffer( + final ByteBuffer buffer, + final ByteOrder byteOrder, + ShapeShiftingColumnarIntsDecodeOptimization overrideOptimization + ) + { + ShapeShiftingColumnData columnData = + new ShapeShiftingColumnData(buffer, (byte) 2, byteOrder, true); + + return new ShapeShiftingColumnarIntsSupplier(columnData, overrideOptimization); + } + + /** + * Supply a {@link ShapeShiftingColumnarInts} + * + * @return + */ + @Override + public ColumnarInts get() + { + Byte2IntMap composition = columnData.getComposition(); + + Byte2ObjectMap> decoders = IntCodecs.getDecoders( + composition.keySet(), + columnData.getLogValuesPerChunk(), + columnData.getByteOrder() + ); + + ShapeShiftingColumnarIntsDecodeOptimization optimization = + overrideOptimization != null + ? overrideOptimization + : ShapeShiftingColumnarIntsDecodeOptimization.fromComposition(columnData, decoders); + + switch (optimization) { + case BLOCK: + return new ShapeShiftingBlockColumnarInts(columnData, decoders); + case MIXED: + default: + return new ShapeShiftingColumnarInts(columnData, decoders); + } + } + + @Override + public long getSerializedSize() throws IOException + { + return columnData.getBaseBuffer().remaining(); + } + + @Override + public void writeTo( + final WritableByteChannel channel, + final FileSmoosher smoosher + ) throws IOException + { + // todo: idk + // ByteBuffer intToBytesHelperBuffer = ByteBuffer.allocate(Integer.BYTES).order(columnData.getByteOrder()); + + // ShapeShiftingColumnSerializer.writeShapeShiftHeader( + // channel, + // intToBytesHelperBuffer, + // ShapeShiftingColumnarInts.VERSION, + // columnData.getNumChunks(), + // columnData.getNumValues(), + // columnData.getLogValuesPerChunk(), + // columnData.getCompositionSize(), + // columnData.getOffsetsSize() + // ); + channel.write(columnData.getBaseBuffer()); + } + + public enum ShapeShiftingColumnarIntsDecodeOptimization + { + MIXED, + BLOCK; + + public static ShapeShiftingColumnarIntsDecodeOptimization fromComposition( + ShapeShiftingColumnData columnData, + Byte2ObjectMap> decoders + ) + { + int numDirectAccess = 0; + int preferDirectAccess = 0; + final Byte2IntMap composition = columnData.getComposition(); + for (Byte2ObjectMap.Entry> intDecoderEntry : decoders.byte2ObjectEntrySet()) { + final FormDecoder intDecoder = intDecoderEntry.getValue(); + if (intDecoder instanceof DirectFormDecoder) { + final int count = composition.get(intDecoderEntry.getByteKey()); + numDirectAccess += count; + if (!(intDecoder instanceof ConstantFormDecoder)) { + preferDirectAccess += count; + } + } + } + + if (preferDirectAccess == 0) { + log.info( + "Using block optimized strategy, %d:%d have random access, %d prefer random access", + numDirectAccess, + columnData.getNumChunks(), + preferDirectAccess + ); + return BLOCK; + } else { + log.info( + "Using mixed access strategy, %d:%d have random access, %d prefer random access", + numDirectAccess, + columnData.getNumChunks(), + preferDirectAccess + ); + return MIXED; + } + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/SingleValueColumnarIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/SingleValueColumnarIntsSerializer.java index 5f4fad508148..e5453f5ca3fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/SingleValueColumnarIntsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/SingleValueColumnarIntsSerializer.java @@ -24,7 +24,7 @@ /** * Serializer that produces {@link ColumnarInts}. */ -public abstract class SingleValueColumnarIntsSerializer implements ColumnarIntsSerializer +public interface SingleValueColumnarIntsSerializer extends ColumnarIntsSerializer { - public abstract void addValue(int val) throws IOException; + void addValue(int val) throws IOException; } diff --git a/processing/src/main/java/org/apache/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java index f6690293012d..7cfec5063212 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/V3CompressedVSizeColumnarMultiIntsSerializer.java @@ -31,7 +31,7 @@ /** * Streams array of integers out in the binary format described by {@link V3CompressedVSizeColumnarMultiIntsSupplier} */ -public class V3CompressedVSizeColumnarMultiIntsSerializer extends ColumnarMultiIntsSerializer +public class V3CompressedVSizeColumnarMultiIntsSerializer implements ColumnarMultiIntsSerializer { private static final byte VERSION = V3CompressedVSizeColumnarMultiIntsSupplier.VERSION; diff --git a/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarIntsSerializer.java index cc59faf5e8a4..ba6a3db66b17 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarIntsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarIntsSerializer.java @@ -35,7 +35,7 @@ /** * Streams integers out in the binary format described by {@link VSizeColumnarInts} */ -public class VSizeColumnarIntsSerializer extends SingleValueColumnarIntsSerializer +public class VSizeColumnarIntsSerializer implements SingleValueColumnarIntsSerializer { private static final byte VERSION = VSizeColumnarInts.VERSION; diff --git a/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarMultiIntsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarMultiIntsSerializer.java index 088f2b5c9ee7..24a82ec49527 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarMultiIntsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/VSizeColumnarMultiIntsSerializer.java @@ -34,7 +34,7 @@ /** * Streams arrays of objects out in the binary format described by {@link VSizeColumnarMultiInts}. */ -public class VSizeColumnarMultiIntsSerializer extends ColumnarMultiIntsSerializer +public class VSizeColumnarMultiIntsSerializer implements ColumnarMultiIntsSerializer { private static final byte VERSION = 0x1; diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ArrayFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ArrayFormDecoder.java new file mode 100644 index 000000000000..f7df7487542c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ArrayFormDecoder.java @@ -0,0 +1,26 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumn; + +public interface ArrayFormDecoder extends FormDecoder +{ +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormDecoder.java new file mode 100644 index 000000000000..5816526ee3e5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormDecoder.java @@ -0,0 +1,43 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumn; + +import java.nio.ByteOrder; + +/** + * Common base type for {@link FormDecoder} implementations of any type of {@link ShapeShiftingColumn} + * + * @param + */ +public abstract class BaseFormDecoder implements FormDecoder +{ + protected final byte logValuesPerChunk; + protected final int valuesPerChunk; + protected final ByteOrder byteOrder; + + public BaseFormDecoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + this.logValuesPerChunk = logValuesPerChunk; + this.valuesPerChunk = 1 << logValuesPerChunk; + this.byteOrder = byteOrder; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormEncoder.java new file mode 100644 index 000000000000..4fba56f9c5a1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/BaseFormEncoder.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumnSerializer; + +import java.nio.ByteOrder; + +/** + * Common base type for {@link FormEncoder} implementations of any type of + * {@link ShapeShiftingColumnSerializer} + * + * @param + * @param + */ +public abstract class BaseFormEncoder + implements FormEncoder +{ + protected final byte logValuesPerChunk; + protected final int valuesPerChunk; + protected final ByteOrder byteOrder; + protected final boolean isBigEndian; + + public BaseFormEncoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + this.logValuesPerChunk = logValuesPerChunk; + this.valuesPerChunk = 1 << logValuesPerChunk; + this.byteOrder = byteOrder; + this.isBigEndian = byteOrder.equals(ByteOrder.BIG_ENDIAN); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormDecoder.java new file mode 100644 index 000000000000..bd1be9cbec26 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormDecoder.java @@ -0,0 +1,80 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.ShapeShiftingColumn; +import sun.nio.ch.DirectBuffer; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Generic Shapeshifting form decoder for chunks that are block compressed using any of {@link CompressionStrategy}. + * Data is decompressed to 'decompressedDataBuffer' to be further decoded by another {@link BaseFormDecoder}, + * via calling 'transform' again on the decompressed chunk. + */ +public final class CompressedFormDecoder> + extends BaseFormDecoder +{ + private final byte header; + + public CompressedFormDecoder(byte logValuesPerChunk, ByteOrder byteOrder, byte header) + { + super(logValuesPerChunk, byteOrder); + this.header = header; + } + + @Override + public void transform(TShapeShiftImpl shapeshiftingColumn) + { + final ByteBuffer buffer = shapeshiftingColumn.getBuffer(); + final ByteBuffer decompressed = shapeshiftingColumn.getDecompressedDataBuffer(); + int startOffset = shapeshiftingColumn.getCurrentChunkStartOffset(); + int endOffset = startOffset + shapeshiftingColumn.getCurrentChunkSize(); + decompressed.clear(); + + final CompressionStrategy.Decompressor decompressor = + CompressionStrategy.forId(buffer.get(startOffset++)).getDecompressor(); + + // metadata for inner encoding is stored outside of the compressed values chunk, so set column metadata offsets + // accordingly + final byte chunkCodec = buffer.get(startOffset++); + shapeshiftingColumn.setCurrentChunkStartOffset(startOffset); + FormDecoder innerForm = shapeshiftingColumn.getFormDecoder(chunkCodec); + startOffset += innerForm.getMetadataSize(); + final int size = endOffset - startOffset; + + decompressor.decompress(buffer, startOffset, size, decompressed); + shapeshiftingColumn.setCurrentValueBuffer(decompressed); + shapeshiftingColumn.setCurrentValuesStartOffset(0); + if (decompressed.isDirect()) { + shapeshiftingColumn.setCurrentValuesAddress(((DirectBuffer) decompressed).address()); + } + // transform again, this time using the inner form against the the decompressed buffer + shapeshiftingColumn.transform(innerForm); + } + + @Override + public byte getHeader() + { + return header; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormEncoder.java new file mode 100644 index 000000000000..76ad44dcfa22 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressedFormEncoder.java @@ -0,0 +1,135 @@ +/* + * 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.data.codecs; + +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Generic compression encoder that can wrap {@link CompressibleFormEncoder} to provide any of the compression + * algorithms available in {@link CompressionStrategy} + * + * @param + * @param + */ +public abstract class CompressedFormEncoder + extends BaseFormEncoder +{ + private final CompressibleFormEncoder formEncoder; + private final CompressionStrategy compressionStrategy; + private final CompressionStrategy.Compressor compressor; + private final ByteBuffer uncompressedDataBuffer; + private final ByteBuffer compressedDataBuffer; + private ByteBuffer compressed; + + public CompressedFormEncoder( + byte logValuesPerChunk, + ByteOrder byteOrder, + CompressionStrategy strategy, + CompressibleFormEncoder encoder, + ByteBuffer uncompressedDataBuffer, + ByteBuffer compressedDataBuffer + ) + { + super(logValuesPerChunk, byteOrder); + this.formEncoder = encoder; + this.compressionStrategy = strategy; + this.compressor = compressionStrategy.getCompressor(); + this.compressedDataBuffer = compressedDataBuffer; + this.uncompressedDataBuffer = uncompressedDataBuffer; + } + + @Override + public int getEncodedSize( + TChunk values, + int numValues, + TChunkMetrics metrics + ) throws IOException + { + if (!formEncoder.shouldAttemptCompression(metrics)) { + return Integer.MAX_VALUE; + } + + metrics.setCompressionBufferHolder(formEncoder.getHeader()); + uncompressedDataBuffer.clear(); + compressedDataBuffer.clear(); + formEncoder.encodeToBuffer(uncompressedDataBuffer, values, numValues, metrics); + compressed = compressor.compress(uncompressedDataBuffer, compressedDataBuffer); + // compressionId | inner encoding header | inner encoding metadata | compressed values + return 1 + 1 + formEncoder.getMetadataSize() + compressed.remaining(); + } + + @Override + public double getModifiedEncodedSize( + TChunk values, + int numValues, + TChunkMetrics metrics + ) throws IOException + { + int encodedSize = getEncodedSize(values, numValues, metrics); + switch (metrics.getOptimizationTarget()) { + case FASTER: + return encodedSize * formEncoder.getSpeedModifier(metrics) * 1.30; + case SMALLER: + return encodedSize * formEncoder.getSpeedModifier(metrics); + case FASTBUTSMALLISH: + default: + return encodedSize * formEncoder.getSpeedModifier(metrics) * 1.05; + } + } + + @Override + public void encode( + WriteOutBytes valuesOut, + TChunk values, + int numValues, + TChunkMetrics metrics + ) throws IOException + { + if (metrics.getCompressionBufferHolder() == formEncoder.getHeader()) { + valuesOut.write(new byte[]{compressionStrategy.getId(), formEncoder.getHeader()}); + formEncoder.encodeCompressionMetadata(valuesOut, values, numValues, metrics); + valuesOut.write(compressedDataBuffer); + } else { + uncompressedDataBuffer.clear(); + compressedDataBuffer.clear(); + formEncoder.encodeToBuffer(uncompressedDataBuffer, values, numValues, metrics); + valuesOut.write(new byte[]{compressionStrategy.getId(), formEncoder.getHeader()}); + formEncoder.encodeCompressionMetadata(valuesOut, values, numValues, metrics); + valuesOut.write(compressor.compress(uncompressedDataBuffer, compressedDataBuffer)); + } + } + + @Override + public String getName() + { + return StringUtils.format("%s [%s]", compressionStrategy.toString(), formEncoder.getName()); + } + + public FormEncoder getInnerEncoder() + { + return formEncoder; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressibleFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressibleFormEncoder.java new file mode 100644 index 000000000000..be7b9d5392f2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/CompressibleFormEncoder.java @@ -0,0 +1,100 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumnSerializer; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * A CompressibleFormEncoder extends {@link FormEncoder} to allow composition with a {@link CompressedFormEncoder}, + * to further compress encoded values at the byte level in a value agnostic manner using any + * {@link org.apache.druid.segment.data.CompressionStrategy}. + * + * @param + * @param + */ +public interface CompressibleFormEncoder + extends FormEncoder +{ + /** + * Encode values to a temporary buffer to stage for compression by the + * {@link org.apache.druid.segment.data.CompressionStrategy} in use by {@link CompressedFormEncoder} + * + * @param buffer + * @param values + * @param numValues + * @param metadata + * + * @throws IOException + */ + void encodeToBuffer( + ByteBuffer buffer, + TChunk values, + int numValues, + TChunkMetrics metadata + ) throws IOException; + + /** + * Encode any metadata, not including the encoding header byte, to {@link WriteOutBytes}, which will preceede + * compressed values + * + * @param valuesOut + * @param values + * @param numValues + * @param metrics + * + * @throws IOException + */ + default void encodeCompressionMetadata( + WriteOutBytes valuesOut, + TChunk values, + int numValues, + TChunkMetrics metrics + ) throws IOException + { + } + + /** + * Get size of any encoding metadata, not including header byte. + * + * @return + */ + default int getMetadataSize() + { + return 0; + } + + /** + * To be chill, implementations can suggest that + * {@link ShapeShiftingColumnSerializer} should not attempt compression since it can be very + * expensive in terms of compute time, skipping this encoding for consideration for a block of values. + * + * @param hints + * + * @return + */ + default boolean shouldAttemptCompression(TChunkMetrics hints) + { + return true; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ConstantFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ConstantFormDecoder.java new file mode 100644 index 000000000000..fa0f5be2587b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ConstantFormDecoder.java @@ -0,0 +1,26 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumn; + +public interface ConstantFormDecoder extends FormDecoder +{ +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/DirectFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/DirectFormDecoder.java new file mode 100644 index 000000000000..d3a5012f3247 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/DirectFormDecoder.java @@ -0,0 +1,26 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumn; + +public interface DirectFormDecoder extends FormDecoder +{ +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/FormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/FormDecoder.java new file mode 100644 index 000000000000..3cf883823a5f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/FormDecoder.java @@ -0,0 +1,52 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumn; + +/** + * Interface describing value decoders for {@link ShapeShiftingColumn} implmentations. {@link ShapeShiftingColumn} + * operate on principle of being mutated by {@link FormDecoder} to load a chunk of values, preparing it to be able to + * read row values for indexes that fall within that chunk. + * + * @param + */ +public interface FormDecoder +{ + /** + * Transform {@link ShapeShiftingColumn} to be able to read values for this decoder. + * + * @param column + */ + void transform(TColumn column); + + /** + * Size of any chunk specific metadata stored at the start of a chunk, to calculate offset of values from chunk start + * in underlying buffer + * + * @return + */ + default int getMetadataSize() + { + return 0; + } + + byte getHeader(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/FormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/FormEncoder.java new file mode 100644 index 000000000000..8aaa96aa34bd --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/FormEncoder.java @@ -0,0 +1,99 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.data.ShapeShiftingColumn; +import org.apache.druid.segment.data.ShapeShiftingColumnSerializer; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; + +/** + * Interface describing value encoders for use with {@link ShapeShiftingColumnSerializer} + * + * @param Type of value chunk, i.e. {@code int[]}, {@code long[]}, etc. + * @param Type of {@link FormMetrics} that the encoder cosumes + */ +public interface FormEncoder +{ + /** + * Get size in bytes if the values were encoded with this encoder + * + * @param values + * @param numValues + * @param metrics + * + * @return + * + * @throws IOException + */ + int getEncodedSize( + TChunk values, + int numValues, + TChunkMetrics metrics + ) throws IOException; + + default double getSpeedModifier(TChunkMetrics metrics) + { + return 1.0; + } + + default double getModifiedEncodedSize( + TChunk values, + int numValues, + TChunkMetrics metrics + ) throws IOException + { + return getSpeedModifier(metrics) * getEncodedSize(values, numValues, metrics); + } + + /** + * Encode the values to the supplied {@link WriteOutBytes} + * + * @param valuesOut + * @param values + * @param numValues + * @param metrics + * + * @throws IOException + */ + void encode( + WriteOutBytes valuesOut, + TChunk values, + int numValues, + TChunkMetrics metrics + ) throws IOException; + + /** + * Byte value to write as first byte to indicate the type of encoder used for this chunk. This value must be distinct + * for all encoding/decoding strategies tied to a specific implementation of + * {@link ShapeShiftingColumnSerializer} and {@link ShapeShiftingColumn} + * + * @return + */ + byte getHeader(); + + /** + * Get friendly name of this encoder + * + * @return + */ + String getName(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/FormMetrics.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/FormMetrics.java new file mode 100644 index 000000000000..60f23e5f4a8f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/FormMetrics.java @@ -0,0 +1,91 @@ +/* + * 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.data.codecs; + +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.ShapeShiftingColumnSerializer; + +/** + * Base type for collecting statistics about a block of values for + * {@link ShapeShiftingColumnSerializer} to provide to {@link FormEncoder} implementations to + * make decisions about what encoding to employ. + */ +public abstract class FormMetrics +{ + private IndexSpec.ShapeShiftOptimizationTarget optimizationTarget; + private boolean enableEncoderOptOut; + + private byte compressionBufferHolder = -1; + + public FormMetrics(IndexSpec.ShapeShiftOptimizationTarget optimizationTarget, boolean enableEncoderOptOut) + { + this.optimizationTarget = optimizationTarget; + this.enableEncoderOptOut = enableEncoderOptOut; + } + + /** + * Get {@link IndexSpec.ShapeShiftOptimizationTarget}, useful for {@link FormEncoder} + * implementations to adapt their calculations to the supplied indexing preference + * + * @return + */ + public IndexSpec.ShapeShiftOptimizationTarget getOptimizationTarget() + { + return this.optimizationTarget; + } + + /** + * When multiple 'complete' encoders are being employed, allow encoders which 'think' they will perform poorly for a + * given block to opt out of being used, which in some cases can save expensive calculations + * @return + */ + public boolean isEnableEncoderOptOut() + { + return enableEncoderOptOut; + } + + /** + * Total number of rows processed for this block of values + * + * @return + */ + public abstract int getNumValues(); + + /** + * byte header value of last encoder to use compressed bytebuffer, allowing re-use if encoder is chosen rather than + * recompressing + * + * @return + */ + public byte getCompressionBufferHolder() + { + return compressionBufferHolder; + } + + /** + * Set encoder header as holder of compression buffers + * + * @param encoder + */ + public void setCompressionBufferHolder(byte encoder) + { + this.compressionBufferHolder = encoder; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BaseIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BaseIntFormEncoder.java new file mode 100644 index 000000000000..a2795001441e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BaseIntFormEncoder.java @@ -0,0 +1,99 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.codecs.BaseFormEncoder; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Base type for {@link IntFormEncoder} implementations, provides int to byte helper methods + */ +abstract class BaseIntFormEncoder extends BaseFormEncoder implements IntFormEncoder +{ + private final ByteBuffer intToBytesHelperBuffer; + + BaseIntFormEncoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + intToBytesHelperBuffer = ByteBuffer.allocate(Integer.BYTES).order(byteOrder); + } + + /** + * Write integer value to helper buffer + * + * @param n + * + * @return + */ + protected final ByteBuffer toBytes(final int n) + { + intToBytesHelperBuffer.putInt(0, n); + intToBytesHelperBuffer.rewind(); + return intToBytesHelperBuffer; + } + + /** + * Write integer {@param value} byte packed into {@param numBytes} bytes to {@link WriteOutBytes} output + * + * @param valuesOut + * @param numBytes + * @param value + * + * @throws IOException + */ + final void writeOutValue(WriteOutBytes valuesOut, int numBytes, int value) throws IOException + { + intToBytesHelperBuffer.putInt(0, value); + intToBytesHelperBuffer.position(0); + if (isBigEndian) { + valuesOut.write(intToBytesHelperBuffer.array(), Integer.BYTES - numBytes, numBytes); + } else { + valuesOut.write(intToBytesHelperBuffer.array(), 0, numBytes); + } + } + + /** + * Write integer {@param value} byte packed into {@param numBytes} bytes to {@link ByteBuffer} output + * + * @param valuesOut + * @param numBytes + * @param value + */ + final void writeOutValue(ByteBuffer valuesOut, int numBytes, int value) + { + intToBytesHelperBuffer.putInt(0, value); + intToBytesHelperBuffer.position(0); + if (isBigEndian) { + valuesOut.put(intToBytesHelperBuffer.array(), Integer.BYTES - numBytes, numBytes); + } else { + valuesOut.put(intToBytesHelperBuffer.array(), 0, numBytes); + } + } + + @FunctionalInterface + interface WriteOutFunction + { + void write(int value) throws IOException; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormDecoder.java new file mode 100644 index 000000000000..435e44d15f53 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormDecoder.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.data.codecs.ints; + +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.BaseFormDecoder; +import org.apache.druid.segment.data.codecs.DirectFormDecoder; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Byte packed integer decoder based on + * {@link org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier} implementations + * + * layout: + * | header: IntCodecs.BYTEPACK (byte) | numBytes (byte) | encoded values (numValues * numBytes) | + */ +public final class BytePackedIntFormDecoder extends BaseFormDecoder + implements DirectFormDecoder +{ + public static final int BIG_ENDIAN_INT_24_SHIFT = Integer.SIZE - 24; + public static final int LITTLE_ENDIAN_INT_24_MASK = (int) ((1L << 24) - 1); + + public BytePackedIntFormDecoder(final byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + /** + * Eagerly get all values into value array of shapeshifting int column + * + * @param columnarInts + */ + @Override + public void transform(ShapeShiftingColumnarInts columnarInts) + { + // metadata is always in base buffer at current chunk start offset + final ByteBuffer metaBuffer = columnarInts.getBuffer(); + final int metaOffset = columnarInts.getCurrentChunkStartOffset(); + final byte numBytes = metaBuffer.get(metaOffset); + columnarInts.setCurrentBytesPerValue(numBytes); + } + + @Override + public byte getHeader() + { + return IntCodecs.BYTEPACK; + } + + @Override + public int getMetadataSize() + { + return 1; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormEncoder.java new file mode 100644 index 000000000000..92c2d586d6ff --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/BytePackedIntFormEncoder.java @@ -0,0 +1,142 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Byte packing integer encoder based on {@link org.apache.druid.segment.data.CompressedVSizeColumnarIntsSerializer}. This + * encoder is a {@link CompressibleIntFormEncoder} and can be compressed with any + * {@link org.apache.druid.segment.data.CompressionStrategy}. + * + * layout: + * | header: IntCodecs.BYTEPACK (byte) | numBytes (byte) | encoded values (numValues * numBytes) | + */ +public class BytePackedIntFormEncoder extends CompressibleIntFormEncoder +{ + public BytePackedIntFormEncoder(final byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + // todo: oh hey, it's me.. ur copy pasta + public static byte getNumBytesForMax(int maxValue) + { + if (maxValue < 0) { + throw new IAE("maxValue[%s] must be positive", maxValue); + } + if (maxValue <= 0xFF) { + return 1; + } else if (maxValue <= 0xFFFF) { + return 2; + } else if (maxValue <= 0xFFFFFF) { + return 3; + } + return 4; + } + + @Override + public int getEncodedSize( + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + final int numBytes = getNumBytesForMax(metrics.getMaxValue()); + return (numValues * numBytes) + Integer.BYTES - numBytes; + } + + @Override + public void encode( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) throws IOException + { + final byte numBytes = getNumBytesForMax(metrics.getMaxValue()); + valuesOut.write(new byte[]{numBytes}); + WriteOutFunction writer = (value) -> writeOutValue(valuesOut, numBytes, value); + encodeValues(writer, values, numValues); + valuesOut.write(new byte[Integer.BYTES - numBytes]); + } + + @Override + public void encodeToBuffer( + ByteBuffer buffer, + int[] values, + int numValues, + IntFormMetrics metadata + ) throws IOException + { + final byte numBytes = BytePackedIntFormEncoder.getNumBytesForMax(metadata.getMaxValue()); + WriteOutFunction writer = (value) -> writeOutValue(buffer, numBytes, value); + encodeValues(writer, values, numValues); + buffer.put(new byte[Integer.BYTES - numBytes]); + buffer.flip(); + } + + @Override + public void encodeCompressionMetadata( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) throws IOException + { + final byte numBytes = getNumBytesForMax(metrics.getMaxValue()); + valuesOut.write(new byte[]{numBytes}); + } + + @Override + public int getMetadataSize() + { + return 1; + } + + + private void encodeValues( + WriteOutFunction writer, + int[] values, + int numValues + ) throws IOException + { + for (int i = 0; i < numValues; i++) { + writer.write(values[i]); + } + } + + @Override + public byte getHeader() + { + return IntCodecs.BYTEPACK; + } + + @Override + public String getName() + { + return "bytepack"; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressedIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressedIntFormEncoder.java new file mode 100644 index 000000000000..f44a1fd5d5e8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressedIntFormEncoder.java @@ -0,0 +1,57 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.data.codecs.CompressedFormEncoder; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Int typed {@link CompressedFormEncoder} for compressing a {@link CompressibleIntFormEncoder} with a + * {@link CompressionStrategy}. The inner encoder encodes to a temporary data buffer, and then data is compressed to + * the compression buffer. + * + * layout: + * | header: IntCodecs.COMPRESSED (byte) | compressed data (compressedDataBuffer.remaining()) | + */ +public final class CompressedIntFormEncoder extends CompressedFormEncoder + implements IntFormEncoder +{ + public CompressedIntFormEncoder( + byte logValuesPerChunk, + ByteOrder byteOrder, + CompressionStrategy strategy, + CompressibleIntFormEncoder encoder, + ByteBuffer uncompressedDataBuffer, + ByteBuffer compressedDataBuffer + ) + { + super(logValuesPerChunk, byteOrder, strategy, encoder, uncompressedDataBuffer, compressedDataBuffer); + } + + + @Override + public byte getHeader() + { + return IntCodecs.COMPRESSED; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressibleIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressibleIntFormEncoder.java new file mode 100644 index 000000000000..1be31b162389 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/CompressibleIntFormEncoder.java @@ -0,0 +1,37 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.codecs.CompressibleFormEncoder; + +import java.nio.ByteOrder; + +/** + * Base type for {@link IntFormEncoder} implementations which also implement {@link CompressibleFormEncoder} and are + * compressible with a {@link org.apache.druid.segment.data.CompressionStrategy} + */ +public abstract class CompressibleIntFormEncoder extends BaseIntFormEncoder + implements CompressibleFormEncoder +{ + CompressibleIntFormEncoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormDecoder.java new file mode 100644 index 000000000000..2234ba9bd7f1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormDecoder.java @@ -0,0 +1,57 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.BaseFormDecoder; +import org.apache.druid.segment.data.codecs.ConstantFormDecoder; +import org.apache.druid.segment.data.codecs.DirectFormDecoder; + +import java.nio.ByteOrder; + +/** + * Decoder used if all values are the same within a chunk are constant. + * + * layout: + * | header: IntCodecs.CONSTANT (byte) | constant value (int) | + */ +public final class ConstantIntFormDecoder extends BaseFormDecoder + implements ConstantFormDecoder, DirectFormDecoder +{ + public ConstantIntFormDecoder(final byte logValuesPerChunk, final ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + @Override + public void transform(ShapeShiftingColumnarInts columnarInts) + { + final int startOffset = columnarInts.getCurrentValuesStartOffset(); + final int currentConstant = columnarInts.getCurrentValueBuffer().getInt(startOffset); + columnarInts.setCurrentBytesPerValue(0); + columnarInts.setCurrentConstant(currentConstant); + } + + @Override + public byte getHeader() + { + return IntCodecs.CONSTANT; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormEncoder.java new file mode 100644 index 000000000000..a076d331bddf --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ConstantIntFormEncoder.java @@ -0,0 +1,91 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteOrder; + +/** + * Encoding optimization used if all values are the same within a chunk are constant, using 5 bytes total, including + * the header. + * + * layout: + * | header: IntCodecs.CONSTANT (byte) | constant value (int) | + */ +public class ConstantIntFormEncoder extends BaseIntFormEncoder +{ + public ConstantIntFormEncoder(final byte logValuesPerChunk, final ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + @Override + public int getEncodedSize( + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + if (metrics.isConstant()) { + return Integer.BYTES; + } + return Integer.MAX_VALUE; + } + + @Override + public double getModifiedEncodedSize( + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + if (metrics.isConstant()) { + // count as 1 byte for sake of comparison, iow, never replace zero, but prefer this over 2 bpv rle + return 1; + } + return Integer.MAX_VALUE; + } + + + @Override + public void encode( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) throws IOException + { + valuesOut.write(toBytes(metrics.getMaxValue())); + } + + @Override + public byte getHeader() + { + return IntCodecs.CONSTANT; + } + + @Override + public String getName() + { + return "constant"; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntCodecs.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntCodecs.java new file mode 100644 index 000000000000..9c45b97df55b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntCodecs.java @@ -0,0 +1,81 @@ +/* + * 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.data.codecs.ints; + +import it.unimi.dsi.fastutil.bytes.Byte2ObjectArrayMap; +import it.unimi.dsi.fastutil.bytes.Byte2ObjectMap; +import it.unimi.dsi.fastutil.bytes.ByteSet; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.CompressedFormDecoder; +import org.apache.druid.segment.data.codecs.FormDecoder; + +import java.nio.ByteOrder; + +public class IntCodecs +{ + public static final byte ZERO = 0x00; + public static final byte CONSTANT = 0x01; + public static final byte UNENCODED = 0x02; + public static final byte BYTEPACK = 0x03; + public static final byte RLE_BYTEPACK = 0x04; + public static final byte COMPRESSED = 0x05; + public static final byte FASTPFOR = 0x06; + + + public static Byte2ObjectMap> getDecoders( + ByteSet composition, + byte logValuesPerChunk, + ByteOrder byteOrder + ) + { + Byte2ObjectArrayMap compositionMap = new Byte2ObjectArrayMap(composition.size()); + for (byte b : composition) { + compositionMap.put(b, getDecoder(b, logValuesPerChunk, byteOrder)); + } + return compositionMap; + } + + public static FormDecoder getDecoder( + byte header, + byte logValuesPerChunk, + ByteOrder byteOrder + ) + { + switch (header) { + case ZERO: + return new ZeroIntFormDecoder(logValuesPerChunk, byteOrder); + case CONSTANT: + return new ConstantIntFormDecoder(logValuesPerChunk, byteOrder); + case UNENCODED: + return new UnencodedIntFormDecoder(logValuesPerChunk, byteOrder); + case BYTEPACK: + return new BytePackedIntFormDecoder(logValuesPerChunk, byteOrder); + case RLE_BYTEPACK: + return new RunLengthBytePackedIntFormDecoder(logValuesPerChunk, byteOrder); + case FASTPFOR: + return new LemireIntFormDecoder(logValuesPerChunk, IntCodecs.FASTPFOR, byteOrder); + case COMPRESSED: + return new CompressedFormDecoder<>(logValuesPerChunk, byteOrder, IntCodecs.COMPRESSED); + } + + throw new RE("Unknown decoder[%d]", (int) header); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormEncoder.java new file mode 100644 index 000000000000..54d1ea10abc1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormEncoder.java @@ -0,0 +1,26 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.codecs.FormEncoder; + +public interface IntFormEncoder extends FormEncoder +{ +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormMetrics.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormMetrics.java new file mode 100644 index 000000000000..304d2bf70cf1 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/IntFormMetrics.java @@ -0,0 +1,168 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.ShapeShiftingColumnarIntsSerializer; +import org.apache.druid.segment.data.codecs.FormMetrics; + +/** + * Aggregates statistics about blocks of integer values, such as total number of values processed, minimum and maximum + * values encountered, if the chunk is constant or all zeros, and various facts about data which is repeated more than + * twice ('runs') including number of distinct runs, longest run length, and total number of runs. This information is + * collected by {@link ShapeShiftingColumnarIntsSerializer} which processing row values, and is + * provided to {@link IntFormEncoder} implementations to do anything from estimate encoded size to influencing how + * {@link ShapeShiftingColumnarIntsSerializer} decides whether or not to employ that particular + * encoding. + */ +public class IntFormMetrics extends FormMetrics +{ + private int minValue = Integer.MAX_VALUE; + private int maxValue = Integer.MIN_VALUE; + private int numRunValues = 0; + private int numDistinctRuns = 0; + private int longestRun; + private int currentRun; + private int previousValue; + private int numValues = 0; + private boolean isFirstValue = true; + + public IntFormMetrics(IndexSpec.ShapeShiftOptimizationTarget target, boolean enableEncoderOptOut) + { + super(target, enableEncoderOptOut); + } + + @Override + public int getNumValues() + { + return numValues; + } + + /** + * Minimum integer value encountered in the block of values + * + * @return + */ + public int getMinValue() + { + return minValue; + } + + /** + * Maximum integer value encountered in the block of values + * + * @return + */ + public int getMaxValue() + { + return maxValue; + } + + /** + * Total count of values which are part of a 'run', or a repitition of a value 3 or more times + * + * @return + */ + public int getNumRunValues() + { + return numRunValues; + } + + /** + * Count of distinct of 'runs', or values which are repeated more than 2 times + * + * @return + */ + public int getNumDistinctRuns() + { + return numDistinctRuns; + } + + /** + * Count of longest continuous sequence of repeated values + * + * @return + */ + public int getLongestRun() + { + return longestRun; + } + + /** + * All block values are a constant + * + * @return + */ + public boolean isConstant() + { + return minValue == maxValue; + } + + /** + * All block values are zero + * + * @return + */ + public boolean isZero() + { + return minValue == 0 && minValue == maxValue; + } + + + /** + * This method is called for every {@link ShapeShiftingColumnarIntsSerializer#addValue(int)} to + * aggregate details about a chunk of values. + * + * @param val row value + */ + public void processNextRow(int val) + { + if (isFirstValue) { + isFirstValue = false; + previousValue = val; + currentRun = 1; + longestRun = 1; + } else { + if (val == previousValue) { + currentRun++; + if (currentRun > 2) { + numRunValues++; + } + } else { + previousValue = val; + if (currentRun > 2) { + numDistinctRuns++; + } + currentRun = 1; + } + } + + if (currentRun > longestRun) { + longestRun = currentRun; + } + if (val < minValue) { + minValue = val; + } + if (val > maxValue) { + maxValue = val; + } + numValues++; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormDecoder.java new file mode 100644 index 000000000000..a088f6d1002e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormDecoder.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.data.codecs.ints; + +import me.lemire.integercompression.IntWrapper; +import me.lemire.integercompression.SkippableIntegerCODEC; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.segment.CompressedPools; +import org.apache.druid.segment.data.ShapeShiftingColumn; +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.ArrayFormDecoder; +import org.apache.druid.segment.data.codecs.BaseFormDecoder; +import sun.misc.Unsafe; +import sun.nio.ch.DirectBuffer; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Integer form decoder using {@link JavaFastPFOR}. Any + * {@link SkippableIntegerCODEC} will work, but currently only {@link me.lemire.integercompression.FastPFOR} is + * setup as a known encoding in {@link IntCodecs} as {@link IntCodecs#FASTPFOR}. + * Eagerly decodes all values to {@link ShapeShiftingColumnarInts#decodedValues}. + * + * layout: + * | header (byte) | encoded values (numOutputInts * Integer.BYTES) | + */ +public final class LemireIntFormDecoder extends BaseFormDecoder + implements ArrayFormDecoder +{ + private static final Unsafe unsafe = ShapeShiftingColumn.getTheUnsafe(); + private final NonBlockingPool codecPool; + private final byte header; + + public LemireIntFormDecoder( + byte logValuesPerChunk, + byte header, + ByteOrder byteOrder + ) + { + super(logValuesPerChunk, byteOrder); + this.header = header; + this.codecPool = CompressedPools.getShapeshiftLemirePool(header, logValuesPerChunk); + } + + /** + * Eagerly decode all values into value array of shapeshifting int column + * + * @param columnarInts + */ + @Override + public void transform(ShapeShiftingColumnarInts columnarInts) + { + final int numValues = columnarInts.getCurrentChunkNumValues(); + final int startOffset = columnarInts.getCurrentValuesStartOffset(); + final int endOffset = startOffset + columnarInts.getCurrentChunkSize(); + final ByteBuffer buffer = columnarInts.getCurrentValueBuffer(); + final int[] decodedChunk = columnarInts.getDecodedValues(); + + final int chunkSizeBytes = endOffset - startOffset; + + // todo: needed? + //CHECKSTYLE.OFF: Regexp +// if (chunkSizeBytes % Integer.BYTES != 0) { +// throw new ISE( +// "Expected to read a whole number of integers, but got[%d] to [%d] for chunk", +// startOffset, +// endOffset +// ); +// } + //CHECKSTYLE.ON: Regexp + + // Copy chunk into an int array. + final int chunkSizeAsInts = chunkSizeBytes >> 2; + + try ( + ResourceHolder tmpHolder = CompressedPools.getShapeshiftIntsEncodedValuesArray(logValuesPerChunk); + ResourceHolder codecHolder = codecPool.take() + ) { + final int[] tmp = tmpHolder.get(); + final SkippableIntegerCODEC codec = codecHolder.get(); + + if (buffer.isDirect() && byteOrder.equals(ByteOrder.nativeOrder())) { + long addr = ((DirectBuffer) buffer).address() + startOffset; + for (int i = 0; i < chunkSizeAsInts; i++, addr += Integer.BYTES) { + tmp[i] = unsafe.getInt(addr); + } + } else { + for (int i = 0, bufferPos = startOffset; i < chunkSizeAsInts; i += 1, bufferPos += Integer.BYTES) { + tmp[i] = buffer.getInt(bufferPos); + } + } + + // Decompress the chunk. + final IntWrapper inPos = new IntWrapper(0); + final IntWrapper outPos = new IntWrapper(0); + + // this will unpack encodedValuesTmp to decodedValues + codec.headlessUncompress( + tmp, + inPos, + chunkSizeAsInts, + decodedChunk, + outPos, + numValues + ); + } + + // todo: needed? + // Sanity checks. + //CHECKSTYLE.OFF: Regexp +// if (inPos.get() != chunkSizeAsInts) { +// throw new ISE( +// "Expected to read[%d] ints but actually read[%d]", +// chunkSizeAsInts, +// inPos.get() +// ); +// } +// +// if (outPos.get() != numValues) { +// throw new ISE("Expected to get[%d] ints but actually got[%d]", numValues, outPos.get()); +// } + //CHECKSTYLE.ON: Regexp + } + + @Override + public byte getHeader() + { + return header; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormEncoder.java new file mode 100644 index 000000000000..21dfa28bb98f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/LemireIntFormEncoder.java @@ -0,0 +1,120 @@ +/* + * 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.data.codecs.ints; + +import me.lemire.integercompression.IntWrapper; +import me.lemire.integercompression.SkippableIntegerCODEC; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.CompressedPools; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteOrder; + +/** + * Integer form encoder using {@link JavaFastPFOR}. Any + * {@link SkippableIntegerCODEC} should work, but currently only {@link me.lemire.integercompression.FastPFOR} is + * setup as a known encoding in {@link IntCodecs} as {@link IntCodecs#FASTPFOR}. + * + * layout: + * | header (byte) | encoded values (numOutputInts * Integer.BYTES) | + */ +public final class LemireIntFormEncoder extends BaseIntFormEncoder +{ + private final NonBlockingPool codecPool; + private final byte header; + private final String name; + private int numOutputInts; + + public LemireIntFormEncoder( + byte logValuesPerChunk, + byte header, + String name, + ByteOrder byteOrder + ) + { + super(logValuesPerChunk, byteOrder); + this.header = header; + this.name = name; + this.codecPool = CompressedPools.getShapeshiftLemirePool(header, logValuesPerChunk); + } + + @Override + public int getEncodedSize( + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + try (ResourceHolder tmpHolder = CompressedPools.getShapeshiftIntsEncodedValuesArray(logValuesPerChunk)) { + final int[] encodedValuesTmp = tmpHolder.get(); + numOutputInts = doEncode(values, encodedValuesTmp, numValues); + return numOutputInts * Integer.BYTES; + } + } + + @Override + public void encode( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) throws IOException + { + try (ResourceHolder tmpHolder = CompressedPools.getShapeshiftIntsEncodedValuesArray(logValuesPerChunk)) { + final int[] encodedValuesTmp = tmpHolder.get(); + numOutputInts = doEncode(values, encodedValuesTmp, numValues); + for (int i = 0; i < numOutputInts; i++) { + valuesOut.write(toBytes(encodedValuesTmp[i])); + } + } + } + + @Override + public byte getHeader() + { + return header; + } + + @Override + public String getName() + { + return name; + } + + private int doEncode(int[] values, int[] encodedValuesTmp, final int numValues) + { + final IntWrapper inPos = new IntWrapper(0); + final IntWrapper outPos = new IntWrapper(0); + + try (ResourceHolder codecHolder = codecPool.take()) { + final SkippableIntegerCODEC codec = codecHolder.get(); + codec.headlessCompress(values, inPos, numValues, encodedValuesTmp, outPos); + } + + if (inPos.get() != numValues) { + throw new ISE("Expected to compress[%d] ints, but read[%d]", numValues, inPos.get()); + } + + return outPos.get(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormDecoder.java new file mode 100644 index 000000000000..4d2753c3385c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormDecoder.java @@ -0,0 +1,283 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.ShapeShiftingColumn; +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.ArrayFormDecoder; +import org.apache.druid.segment.data.codecs.BaseFormDecoder; +import sun.misc.Unsafe; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * layout: + * | header: IntCodecs.RLE_BYTEPACK (byte) | numBytes (byte) | encoded values ((2 * numDistinctRuns * numBytes) + (numSingleValues * numBytes)) | + */ +public final class RunLengthBytePackedIntFormDecoder extends BaseFormDecoder + implements ArrayFormDecoder +{ + static final int value_mask_int8 = 0x7F; + static final int value_mask_int16 = 0x7FFF; + static final int value_mask_int24 = 0x7FFFFF; + static final int value_mask_int32 = 0x7FFFFFFF; + + static final int run_mask_int8 = 0x80; + static final int run_mask_int16 = 0x8000; + static final int run_mask_int24 = 0x800000; + static final int run_mask_int32 = 0x80000000; + + private static final Unsafe unsafe = ShapeShiftingColumn.getTheUnsafe(); + + public RunLengthBytePackedIntFormDecoder(final byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + @Override + public void transform(ShapeShiftingColumnarInts columnarInts) + { + final ByteBuffer buffer = columnarInts.getCurrentValueBuffer(); + // metadata is always in base buffer at current chunk start offset + final ByteBuffer metaBuffer = columnarInts.getBuffer(); + final int metaOffset = columnarInts.getCurrentChunkStartOffset(); + final byte numBytes = metaBuffer.get(metaOffset); + final int[] decodedChunk = columnarInts.getDecodedValues(); + final int numValues = columnarInts.getCurrentChunkNumValues(); + final int startOffset = columnarInts.getCurrentValuesStartOffset(); + + if (buffer.isDirect() && byteOrder.equals(ByteOrder.nativeOrder())) { + final long addr = columnarInts.getCurrentValuesAddress(); + decodeIntsUnsafe(addr, numValues, decodedChunk, numBytes, byteOrder); + } else { + decodeIntsBuffer(buffer, startOffset, numValues, decodedChunk, numBytes, byteOrder); + } + } + + @Override + public byte getHeader() + { + return IntCodecs.RLE_BYTEPACK; + } + + @Override + public int getMetadataSize() + { + return 1; + } + + private static void decodeIntsUnsafe( + long addr, + final int numValues, + final int[] decodedValues, + final int bytePerValue, + final ByteOrder byteOrder + ) + { + int runCount; + int runValue; + + final boolean isBigEndian = byteOrder.equals(ByteOrder.BIG_ENDIAN); + final DecodeAddressFunction decode; + final int runMask; + final int valueMask; + switch (bytePerValue) { + case 1: + decode = RunLengthBytePackedIntFormDecoder::decodeInt8Unsafe; + runMask = run_mask_int8; + valueMask = value_mask_int8; + break; + case 2: + decode = RunLengthBytePackedIntFormDecoder::decodeInt16Unsafe; + runMask = run_mask_int16; + valueMask = value_mask_int16; + break; + case 3: + decode = isBigEndian + ? RunLengthBytePackedIntFormDecoder::decodeBigEndianInt24Unsafe + : RunLengthBytePackedIntFormDecoder::decodeInt24Unsafe; + runMask = run_mask_int24; + valueMask = value_mask_int24; + break; + default: + decode = RunLengthBytePackedIntFormDecoder::decodeInt32Unsafe; + runMask = run_mask_int32; + valueMask = value_mask_int32; + break; + } + + for (int i = 0; i < numValues; i++) { + final int nextVal = decode.get(addr); + addr += bytePerValue; + if ((nextVal & runMask) == 0) { + decodedValues[i] = nextVal; + } else { + runCount = nextVal & valueMask; + runValue = decode.get(addr); + addr += bytePerValue; + do { + decodedValues[i] = runValue; + } while (--runCount > 0 && ++i < numValues); + } + } + } + + + private static int decodeInt8Unsafe(long addr) + { + return unsafe.getByte(addr) & 0xFF; + } + + private static int decodeInt16Unsafe(long addr) + { + return unsafe.getShort(addr) & 0xFFFF; + } + + private static int decodeBigEndianInt24Unsafe(long addr) + { + // big-endian: 0x000c0b0a stored 0c 0b 0a XX, read 0x0c0b0aXX >>> 8 + return unsafe.getInt(addr) >>> BytePackedIntFormDecoder.BIG_ENDIAN_INT_24_SHIFT; + } + + private static int decodeInt24Unsafe(long addr) + { + // little-endian: 0x000c0b0a stored 0a 0b 0c XX, read 0xXX0c0b0a & 0x00FFFFFF + return unsafe.getInt(addr) & BytePackedIntFormDecoder.LITTLE_ENDIAN_INT_24_MASK; + } + + private static int decodeInt32Unsafe(long addr) + { + return unsafe.getInt(addr); + } + + public static void decodeIntsBuffer( + ByteBuffer buffer, + final int startOffset, + final int numValues, + final int[] decodedValues, + final int bytePerValue, + final ByteOrder byteOrder + ) + { + + int bufferPosition = startOffset; + int runCount; + int runValue; + + final boolean isBigEndian = byteOrder.equals(ByteOrder.BIG_ENDIAN); + final DecodeBufferFunction decode; + final int runMask; + final int valueMask; + switch (bytePerValue) { + case 1: + decode = RunLengthBytePackedIntFormDecoder::decodeInt8; + runMask = run_mask_int8; + valueMask = value_mask_int8; + break; + case 2: + decode = RunLengthBytePackedIntFormDecoder::decodeInt16; + runMask = run_mask_int16; + valueMask = value_mask_int16; + break; + case 3: + decode = isBigEndian + ? RunLengthBytePackedIntFormDecoder::decodeBigEndianInt24 + : RunLengthBytePackedIntFormDecoder::decodeInt24; + runMask = run_mask_int24; + valueMask = value_mask_int24; + break; + default: + decode = RunLengthBytePackedIntFormDecoder::decodeInt32; + runMask = run_mask_int32; + valueMask = value_mask_int32; + break; + } + + for (int i = 0; i < numValues; i++) { + final int nextVal = decode.get(buffer, bufferPosition); + bufferPosition += bytePerValue; + if ((nextVal & runMask) == 0) { + decodedValues[i] = nextVal; + } else { + runCount = nextVal & valueMask; + runValue = decode.get(buffer, bufferPosition); + bufferPosition += bytePerValue; + do { + decodedValues[i] = runValue; + } while (--runCount > 0 && ++i < numValues); + } + } + } + + private static int decodeInt8( + final ByteBuffer buffer, + final int offset + ) + { + return buffer.get(offset) & 0xFF; + } + + private static int decodeInt16( + final ByteBuffer buffer, + final int offset + ) + { + return buffer.getShort(offset) & 0xFFFF; + } + + private static int decodeBigEndianInt24( + final ByteBuffer buffer, + final int offset + ) + { + // big-endian: 0x000c0b0a stored 0c 0b 0a XX, read 0x0c0b0aXX >>> 8 + return buffer.getInt(offset) >>> BytePackedIntFormDecoder.BIG_ENDIAN_INT_24_SHIFT; + } + + private static int decodeInt24( + final ByteBuffer buffer, + final int offset + ) + { + // little-endian: 0x000c0b0a stored 0a 0b 0c XX, read 0xXX0c0b0a & 0x00FFFFFF + return buffer.getInt(offset) & BytePackedIntFormDecoder.LITTLE_ENDIAN_INT_24_MASK; + } + + private static int decodeInt32( + final ByteBuffer buffer, + final int offset + ) + { + return buffer.getInt(offset); + } + + @FunctionalInterface + public interface DecodeBufferFunction + { + int get(ByteBuffer buffer, int offset); + } + + @FunctionalInterface + public interface DecodeAddressFunction + { + int get(long address); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormEncoder.java new file mode 100644 index 000000000000..a83f83e14c0f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/RunLengthBytePackedIntFormEncoder.java @@ -0,0 +1,265 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Simple run-length encoding implementation which uses a bytepacking strategy similar to + * {@link BytePackedIntFormEncoder}, where the maximum run length and maximum row value are analzyed to choose a + * number of bytes which both the row values and run counts can be encoded, using the high bit to indicate if the + * bytes represent a run or a single value. A run is encoded with 2 values sized with the chosen number of bytes, + * the first with the high bit set and the run length encoded, the 2nd with the value that is repeated. A single + * value is packed into numBytes with the high bit not set. + * + * layout: + * | header: IntCodecs.RLE_BYTEPACK (byte) | numBytes (byte) | encoded values ((2 * numDistinctRuns * numBytes) + (numSingleValues * numBytes)) | + */ +public class RunLengthBytePackedIntFormEncoder extends CompressibleIntFormEncoder +{ + public RunLengthBytePackedIntFormEncoder(final byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + private static int applyRunMask(int runLength, int numBytes) + { + switch (numBytes) { + case 1: + return runLength | RunLengthBytePackedIntFormDecoder.run_mask_int8; + case 2: + return runLength | RunLengthBytePackedIntFormDecoder.run_mask_int16; + case 3: + return runLength | RunLengthBytePackedIntFormDecoder.run_mask_int24; + default: + return runLength | RunLengthBytePackedIntFormDecoder.run_mask_int32; + } + } + + private static byte getNumBytesForMax(int maxValue, int maxRun) + { + if (maxValue < 0) { + throw new IAE("maxValue[%s] must be positive", maxValue); + } + int toConsider = maxValue > maxRun ? maxValue : maxRun; + if (toConsider <= RunLengthBytePackedIntFormDecoder.value_mask_int8) { + return 1; + } else if (toConsider <= RunLengthBytePackedIntFormDecoder.value_mask_int16) { + return 2; + } else if (toConsider <= RunLengthBytePackedIntFormDecoder.value_mask_int24) { + return 3; + } + return 4; + } + + @Override + public int getEncodedSize( + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + return computeSize(metrics); + } + + /** + * Run length speed modifier is dependent on how effective run length encoding is on the data itself if the + * optimization strategy is not {@link IndexSpec.ShapeShiftOptimizationTarget#SMALLER}, since + * decode performance is not great if run count is small, but approaching 1.0 as the values become constant. + * + * @param metrics + * + * @return + */ + @Override + public double getSpeedModifier(IntFormMetrics metrics) + { + // rle is pretty slow when not in a situation where it is appropriate, penalize if big gains are not projected + final byte numBytesBytepack = BytePackedIntFormEncoder.getNumBytesForMax(metrics.getMaxValue()); + final int bytepackSize = numBytesBytepack * metrics.getNumValues(); + final int size = computeSize(metrics); + // don't bother if not smaller than bytepacking + if (size >= bytepackSize) { + return 10.0; + } + double modifier; + switch (metrics.getOptimizationTarget()) { + case SMALLER: + modifier = 1.0; + break; + default: + modifier = (((double) bytepackSize - (double) size)) / (double) bytepackSize; + break; + } + return Math.max(2.0 - modifier, 1.0); + } + + @Override + public void encode( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) throws IOException + { + final byte numBytes = getNumBytesForMax(metrics.getMaxValue(), metrics.getLongestRun()); + valuesOut.write(new byte[]{numBytes}); + + final WriteOutFunction writer = (value) -> writeOutValue(valuesOut, numBytes, value); + + encodeValues(writer, values, numValues, numBytes); + + // pad if int24, it reads values with buffer.getInt and either masks or shifts as appropriate for endian-ness + // other sizes have native read methods and don't require padding + if (numBytes == 3) { + valuesOut.write(new byte[]{0}); + } + } + + @Override + public void encodeToBuffer( + ByteBuffer buffer, + int[] values, + int numValues, + IntFormMetrics metadata + ) throws IOException + { + final byte numBytes = + RunLengthBytePackedIntFormEncoder.getNumBytesForMax(metadata.getMaxValue(), metadata.getLongestRun()); + + final WriteOutFunction writer = (value) -> writeOutValue(buffer, numBytes, value); + + encodeValues(writer, values, numValues, numBytes); + + // pad if int24, it reads values with buffer.getInt and either masks or shifts as appropriate for endian-ness + // other sizes have native read methods and don't require padding + if (numBytes == 3) { + buffer.put((byte) 0); + } + buffer.flip(); + } + + @Override + public void encodeCompressionMetadata( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) throws IOException + { + final byte numBytes = getNumBytesForMax(metrics.getMaxValue(), metrics.getLongestRun()); + valuesOut.write(new byte[]{numBytes}); + } + + @Override + public int getMetadataSize() + { + return 1; + } + + @Override + public boolean shouldAttemptCompression(IntFormMetrics hints) + { + if (!hints.isEnableEncoderOptOut()) { + return true; + } + + // if not very many runs, cheese it out of here since i am expensive-ish + // todo: this is totally scientific. 100%. If we don't have at least 3/4 runs, then bail on trying compression since expensive + if ((hints.getOptimizationTarget() != IndexSpec.ShapeShiftOptimizationTarget.SMALLER) && + (hints.getNumRunValues() < (3 * (hints.getNumValues() / 4)))) { + return false; + } + + return true; + } + + @Override + public byte getHeader() + { + return IntCodecs.RLE_BYTEPACK; + } + + @Override + public String getName() + { + return "rle-bytepack"; + } + + private void encodeValues(WriteOutFunction writer, int[] values, int numValues, int numBytes) throws IOException + { + int runCounter = 1; + + for (int current = 1; current < numValues; current++) { + final int prev = current - 1; + final int next = current + 1; + // if previous value equals current value, we are in a run, continue accumulating + if (values[prev] == values[current]) { + runCounter++; + if (next < numValues) { + continue; + } + } + // if we get here we are either previously encountered a single value, + // or we are at the end of a run and the current value is the start of a new run or a single value, + // so write out the previous value + if (runCounter > 1) { + if (runCounter > 2) { + // if a run, encode with 2 values, the first masked to indicate that it is a run length, followed by the + // value itself + int maskedCounter = RunLengthBytePackedIntFormEncoder.applyRunMask(runCounter, numBytes); + writer.write(maskedCounter); + writer.write(values[prev]); + runCounter = 1; + } else { + // a run of 2 is lame, and no smaller than encoding directly and avoid entering the inner "run" unrolling + // loop during decoding + //CHECKSTYLE.OFF: duplicateLine + writer.write(values[prev]); + writer.write(values[prev]); + //CHECKSTYLE.ON: duplicateLine + runCounter = 1; + } + } else { + // non runs are written directly + writer.write(values[prev]); + } + // write out the last value if not part of a run + if (next == numValues && values[current] != values[prev]) { + writer.write(values[current]); + } + } + } + + private int computeSize(IntFormMetrics metadata) + { + final byte numBytes = getNumBytesForMax(metadata.getMaxValue(), metadata.getLongestRun()); + int size = numBytes * metadata.getNumValues(); + size -= (numBytes * metadata.getNumRunValues()); + size += (2 * numBytes * metadata.getNumDistinctRuns()); + return size; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormDecoder.java new file mode 100644 index 000000000000..7cb4adc448d7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormDecoder.java @@ -0,0 +1,53 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.BaseFormDecoder; +import org.apache.druid.segment.data.codecs.DirectFormDecoder; + +import java.nio.ByteOrder; + +/** + * "Unencoded" decoder, reads full sized integer values from shapeshifting int column read buffer. + * + * layout: + * | header: IntCodecs.UNENCODED (byte) | values (numValues * Integer.BYTES) | + */ +public final class UnencodedIntFormDecoder extends BaseFormDecoder + implements DirectFormDecoder +{ + public UnencodedIntFormDecoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + @Override + public void transform(ShapeShiftingColumnarInts columnarInts) + { + columnarInts.setCurrentBytesPerValue(Integer.BYTES); + } + + @Override + public byte getHeader() + { + return IntCodecs.UNENCODED; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormEncoder.java new file mode 100644 index 000000000000..c3aed71b4dee --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/UnencodedIntFormEncoder.java @@ -0,0 +1,94 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Unencoded integer encoder that writes full sized integer values as is. + * + * layout: + * | header: IntCodecs.UNENCODED (byte) | values (numValues * Integer.BYTES) | + */ +public class UnencodedIntFormEncoder extends CompressibleIntFormEncoder +{ + public UnencodedIntFormEncoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + @Override + public int getEncodedSize( + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + return numValues * Integer.BYTES; + } + + @Override + public void encode( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) throws IOException + { + WriteOutFunction writer = (value) -> valuesOut.write(toBytes(value)); + encodeValues(writer, values, numValues); + } + + @Override + public void encodeToBuffer( + ByteBuffer buffer, + int[] values, + int numValues, + IntFormMetrics metadata + ) throws IOException + { + WriteOutFunction writer = (value) -> buffer.putInt(value); + encodeValues(writer, values, numValues); + buffer.flip(); + } + + @Override + public byte getHeader() + { + return IntCodecs.UNENCODED; + } + + @Override + public String getName() + { + return "unencoded"; + } + + private void encodeValues(WriteOutFunction writer, int[] values, int numValues) throws IOException + { + for (int i = 0; i < numValues; i++) { + writer.write(values[i]); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormDecoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormDecoder.java new file mode 100644 index 000000000000..ef9aa00999f5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormDecoder.java @@ -0,0 +1,60 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.data.ShapeShiftingColumnarInts; +import org.apache.druid.segment.data.codecs.BaseFormDecoder; +import org.apache.druid.segment.data.codecs.ConstantFormDecoder; +import org.apache.druid.segment.data.codecs.DirectFormDecoder; + +import java.nio.ByteOrder; + +/** + * Decoder used if all values are the same within a chunk are zero. + * + * layout: + * | header: IntCodecs.ZERO (byte) | + */ +public final class ZeroIntFormDecoder extends BaseFormDecoder + implements ConstantFormDecoder, DirectFormDecoder +{ + public ZeroIntFormDecoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + /** + * Fill shapeshifting int column chunk values array with zeros + * + * @param columnarInts + */ + @Override + public void transform(ShapeShiftingColumnarInts columnarInts) + { + columnarInts.setCurrentBytesPerValue(0); + columnarInts.setCurrentConstant(0); + } + + @Override + public byte getHeader() + { + return IntCodecs.ZERO; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormEncoder.java b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormEncoder.java new file mode 100644 index 000000000000..5ff57e69fc3a --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/data/codecs/ints/ZeroIntFormEncoder.java @@ -0,0 +1,75 @@ +/* + * 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.data.codecs.ints; + +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.nio.ByteOrder; + +/** + * Encoding optimization used if all values are the same within a chunk are zero, using 1 byte total, including + * the header. + * + * layout: + * | header: IntCodecs.ZERO (byte) | + */ +public class ZeroIntFormEncoder extends BaseIntFormEncoder +{ + public ZeroIntFormEncoder(byte logValuesPerChunk, ByteOrder byteOrder) + { + super(logValuesPerChunk, byteOrder); + } + + @Override + public int getEncodedSize( + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + if (metrics.isZero()) { + return 0; + } + return Integer.MAX_VALUE; + } + + @Override + public void encode( + WriteOutBytes valuesOut, + int[] values, + int numValues, + IntFormMetrics metrics + ) + { + // do nothing! + } + + @Override + public byte getHeader() + { + return IntCodecs.ZERO; + } + + @Override + public String getName() + { + return "zero"; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/generator/ColumnValueGenerator.java b/processing/src/main/java/org/apache/druid/segment/generator/ColumnValueGenerator.java index fbfc1a9f3c15..1f3ef27d4e2e 100644 --- a/processing/src/main/java/org/apache/druid/segment/generator/ColumnValueGenerator.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/ColumnValueGenerator.java @@ -131,6 +131,13 @@ private Object convertType(Object input, ValueType type) ret = Float.parseFloat(input.toString()); } break; + case INT: + if (input instanceof Number) { + ret = ((Number) input).intValue(); + } else { + ret = Integer.parseInt(input.toString()); + } + break; default: throw new UnsupportedOperationException("Unknown data type: " + type); } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index a0a733ec6f62..7718a989b202 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -42,6 +42,7 @@ import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.GenericIndexedWriter; import org.apache.druid.segment.data.ImmutableRTreeObjectStrategy; +import org.apache.druid.segment.data.ShapeShiftingColumnarIntsSupplier; import org.apache.druid.segment.data.V3CompressedVSizeColumnarMultiIntsSupplier; import org.apache.druid.segment.data.VSizeColumnarInts; import org.apache.druid.segment.data.VSizeColumnarMultiInts; @@ -59,7 +60,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde private static final int NO_FLAGS = 0; private static final int STARTING_FLAGS = Feature.NO_BITMAP_INDEX.getMask(); - enum Feature + public enum Feature { MULTI_VALUE, MULTI_VALUE_V3, @@ -76,12 +77,13 @@ public int getMask() } } - enum VERSION + public enum VERSION { UNCOMPRESSED_SINGLE_VALUE, // 0x0 UNCOMPRESSED_MULTI_VALUE, // 0x1 COMPRESSED, // 0x2 - UNCOMPRESSED_WITH_FLAGS; // 0x3 + UNCOMPRESSED_WITH_FLAGS, // 0x3 + SHAPESHIFT; // 0x4 public static VERSION fromByte(byte b) { @@ -196,7 +198,12 @@ public SerializerBuilder withByteOrder(ByteOrder byteOrder) return this; } - public SerializerBuilder withValue(ColumnarIntsSerializer valueWriter, boolean hasMultiValue, boolean compressed) + public SerializerBuilder withValue( + ColumnarIntsSerializer valueWriter, + boolean hasMultiValue, + boolean compressed, + boolean shapeshift + ) { this.valueWriter = valueWriter; if (hasMultiValue) { @@ -208,7 +215,9 @@ public SerializerBuilder withValue(ColumnarIntsSerializer valueWriter, boolean h this.flags |= Feature.MULTI_VALUE.getMask(); } } else { - if (compressed) { + if (shapeshift) { + this.version = VERSION.SHAPESHIFT; + } else if (compressed) { this.version = VERSION.COMPRESSED; } else { this.version = VERSION.UNCOMPRESSED_SINGLE_VALUE; @@ -362,6 +371,8 @@ private WritableSupplier readSingleValuedColumn(VERSION version, B return VSizeColumnarInts.readFromByteBuffer(buffer); case COMPRESSED: return CompressedVSizeColumnarIntsSupplier.fromByteBuffer(buffer, byteOrder); + case SHAPESHIFT: + return ShapeShiftingColumnarIntsSupplier.fromByteBuffer(buffer, byteOrder); default: throw new IAE("Unsupported single-value version[%s]", version); } diff --git a/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java index 8b4a8d3f5313..602627258c15 100644 --- a/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java @@ -87,6 +87,7 @@ public void testDefaultSegmentizerPersist() throws IOException null, null, null, + null, null ), null @@ -112,7 +113,8 @@ public void testCustomSegmentizerPersist() throws IOException null, null, null, - new CustomSegmentizerFactory() + new CustomSegmentizerFactory(), + null ), null ); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java index e9197ac8647a..0c66b1130c49 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java @@ -42,6 +42,37 @@ public void testSerde() throws Exception Assert.assertEquals(CompressionStrategy.LZ4, spec.getDimensionCompression()); Assert.assertEquals(CompressionStrategy.LZF, spec.getMetricCompression()); Assert.assertEquals(CompressionFactory.LongEncodingStrategy.AUTO, spec.getLongEncoding()); + Assert.assertEquals(IndexSpec.EncodingStrategy.COMPRESSION, spec.getIntEncodingStrategy().getStrategy()); + Assert.assertEquals( + IndexSpec.ShapeShiftOptimizationTarget.FASTBUTSMALLISH, + spec.getIntEncodingStrategy().getOptimizationTarget() + ); + Assert.assertEquals(IndexSpec.ShapeShiftBlockSize.LARGE, spec.getIntEncodingStrategy().getBlockSize()); + + Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class)); + } + + + @Test + public void testSerdeShapeshift() throws Exception + { + final ObjectMapper objectMapper = new DefaultObjectMapper(); + final String json = "{ \"bitmap\" : { \"type\" : \"roaring\" }, \"dimensionCompression\" : \"lz4\", " + + "\"metricCompression\" : \"lzf\", \"longEncoding\" : \"auto\", \"intEncodingStrategy\" : { " + + "\"strategy\" : \"shapeshift\", \"optimizationTarget\" : \"faster\", " + + "\"blockSize\" : \"small\"}}"; + + final IndexSpec spec = objectMapper.readValue(json, IndexSpec.class); + Assert.assertEquals(new RoaringBitmapSerdeFactory(null), spec.getBitmapSerdeFactory()); + Assert.assertEquals(CompressionStrategy.LZ4, spec.getDimensionCompression()); + Assert.assertEquals(CompressionStrategy.LZF, spec.getMetricCompression()); + Assert.assertEquals(CompressionFactory.LongEncodingStrategy.AUTO, spec.getLongEncoding()); + Assert.assertEquals(IndexSpec.EncodingStrategy.SHAPESHIFT, spec.getIntEncodingStrategy().getStrategy()); + Assert.assertEquals( + IndexSpec.ShapeShiftOptimizationTarget.FASTER, + spec.getIntEncodingStrategy().getOptimizationTarget() + ); + Assert.assertEquals(IndexSpec.ShapeShiftBlockSize.SMALL, spec.getIntEncodingStrategy().getBlockSize()); Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class)); } diff --git a/processing/src/test/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerdeTest.java new file mode 100644 index 000000000000..7494f6b003c5 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/data/ShapeShiftingColumnarIntsSerdeTest.java @@ -0,0 +1,357 @@ +/* + * 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.data; + +import com.google.common.collect.Sets; +import com.google.common.primitives.Longs; +import it.unimi.dsi.fastutil.ints.IntArrays; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.guava.CloseQuietly; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.data.codecs.CompressedFormEncoder; +import org.apache.druid.segment.data.codecs.ints.BytePackedIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.IntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.LemireIntFormEncoder; +import org.apache.druid.segment.data.codecs.ints.RunLengthBytePackedIntFormEncoder; +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +@RunWith(Parameterized.class) +public class ShapeShiftingColumnarIntsSerdeTest +{ + + @Parameterized.Parameters(name = "{index}: blockSize={0}, optimizationTarget={1}, byteOrder={2}, encoderSet={3}") + public static List blockSizesOptimizationTargetsAndByteOrders() + { + Set> combinations = Sets.cartesianProduct( + Sets.newHashSet(IndexSpec.ShapeShiftBlockSize.values()), + Sets.newHashSet(IndexSpec.ShapeShiftOptimizationTarget.values()), + Sets.newHashSet(ByteOrder.LITTLE_ENDIAN, ByteOrder.BIG_ENDIAN), + Sets.newHashSet("bytepack", "rle-bytepack", "fastpfor", "lz4-bytepack", "lz4-rle-bytepack", "lz4", "default") + ); + + return combinations.stream() + .map(input -> new Object[]{input.get(0), input.get(1), input.get(2), input.get(3)}) + .collect(Collectors.toList()); + } + + private final IndexSpec.ShapeShiftBlockSize blockSize; + private final IndexSpec.ShapeShiftOptimizationTarget optimizationTarget; + private final ByteOrder byteOrder; + private final String encoderSet; + + private Closer closer; + private ColumnarInts columnarInts; + private ShapeShiftingColumnarIntsSupplier supplier; + private int[] vals; + + public ShapeShiftingColumnarIntsSerdeTest( + IndexSpec.ShapeShiftBlockSize blockSize, + IndexSpec.ShapeShiftOptimizationTarget optimizationTarget, + ByteOrder byteOrder, + String encoderSet + ) + { + this.blockSize = blockSize; + this.optimizationTarget = optimizationTarget; + this.byteOrder = byteOrder; + this.encoderSet = encoderSet; + } + + @Before + public void setUp() + { + closer = Closer.create(); + CloseQuietly.close(columnarInts); + columnarInts = null; + supplier = null; + vals = null; + } + + @After + public void tearDown() throws Exception + { + columnarInts.close(); + closer.close(); + } + + @Test + public void testFidelity() throws Exception + { + int intChunkSize = 1 << (blockSize.getLogBlockSize() - 2); + + seedAndEncodeData(10 * intChunkSize); + + assertIndexMatchesVals(); + } + + + // cargo cult test, this guy is everywhere so why not here too + @Test + public void testConcurrentThreadReads() throws Exception + { + int intChunkSize = 1 << (blockSize.getLogBlockSize() - 2); + seedAndEncodeData(3 * intChunkSize); + + final AtomicReference reason = new AtomicReference<>("none"); + + final int numRuns = 1000; + final CountDownLatch startLatch = new CountDownLatch(1); + final CountDownLatch stopLatch = new CountDownLatch(2); + final AtomicBoolean failureHappened = new AtomicBoolean(false); + new Thread(() -> { + try { + startLatch.await(); + } + catch (InterruptedException e) { + failureHappened.set(true); + reason.set("interrupt."); + stopLatch.countDown(); + return; + } + + try { + for (int i = 0; i < numRuns; ++i) { + for (int j = 0, size = columnarInts.size(); j < size; ++j) { + final long val = vals[j]; + final long indexedVal = columnarInts.get(j); + if (Longs.compare(val, indexedVal) != 0) { + failureHappened.set(true); + reason.set(StringUtils.format("Thread1[%d]: %d != %d", j, val, indexedVal)); + stopLatch.countDown(); + return; + } + } + } + } + catch (Exception e) { + e.printStackTrace(); + failureHappened.set(true); + reason.set(e.getMessage()); + } + + stopLatch.countDown(); + }).start(); + + final ColumnarInts columnarInts2 = supplier.get(); + try { + new Thread(() -> { + try { + startLatch.await(); + } + catch (InterruptedException e) { + stopLatch.countDown(); + return; + } + + try { + for (int i = 0; i < numRuns; ++i) { + for (int j = columnarInts2.size() - 1; j >= 0; --j) { + final long val = vals[j]; + final long indexedVal = columnarInts2.get(j); + if (Longs.compare(val, indexedVal) != 0) { + failureHappened.set(true); + reason.set(StringUtils.format("Thread2[%d]: %d != %d", j, val, indexedVal)); + stopLatch.countDown(); + return; + } + } + } + } + catch (Exception e) { + e.printStackTrace(); + reason.set(e.getMessage()); + failureHappened.set(true); + } + + stopLatch.countDown(); + }).start(); + + startLatch.countDown(); + + stopLatch.await(); + } + finally { + CloseQuietly.close(columnarInts2); + } + + if (failureHappened.get()) { + Assert.fail("Failure happened. Reason: " + reason.get()); + } + } + + private void serializeAndGetSupplier() throws IOException + { + SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium(); + + final SegmentWriteOutMedium segmentWriteOutMedium = new OnHeapMemorySegmentWriteOutMedium(); + IntFormEncoder[] encoders = ShapeShiftingColumnarIntsSerializer.getDefaultIntFormEncoders( + blockSize, + CompressionStrategy.LZ4, + writeOutMedium.getCloser(), + byteOrder + ); + + List encoderList = Arrays.asList(encoders); + + // todo: probably a better way to do this... + switch (encoderSet) { + case "bytepack": + encoders = encoderList.stream() + .filter(e -> e instanceof BytePackedIntFormEncoder) + .toArray(IntFormEncoder[]::new); + break; + case "rle-bytepack": + encoders = encoderList.stream() + .filter(e -> e instanceof RunLengthBytePackedIntFormEncoder) + .toArray(IntFormEncoder[]::new); + break; + case "fastpfor": + encoders = encoderList.stream() + .filter(e -> e instanceof LemireIntFormEncoder) + .toArray(IntFormEncoder[]::new); + break; + case "lz4-bytepack": + encoders = + encoderList.stream() + .filter(e -> e instanceof CompressedFormEncoder && + ((CompressedFormEncoder) e).getInnerEncoder() instanceof BytePackedIntFormEncoder + ).toArray(IntFormEncoder[]::new); + break; + case "lz4-rle-bytepack": + encoders = + encoderList.stream() + .filter(e -> e instanceof CompressedFormEncoder && + ((CompressedFormEncoder) e).getInnerEncoder() instanceof RunLengthBytePackedIntFormEncoder + ).toArray(IntFormEncoder[]::new); + break; + case "lz4": + encoders = + encoderList.stream().filter(e -> e instanceof CompressedFormEncoder).toArray(IntFormEncoder[]::new); + break; + } + + ShapeShiftingColumnarIntsSerializer serializer = new ShapeShiftingColumnarIntsSerializer( + segmentWriteOutMedium, + encoders, + optimizationTarget, + blockSize, + byteOrder + ); + serializer.open(); + for (int val : vals) { + serializer.addValue(val); + } + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.writeTo(Channels.newChannel(baos), null); + + writeOutMedium.close(); + final byte[] bytes = baos.toByteArray(); + Assert.assertEquals(serializer.getSerializedSize(), bytes.length); + + supplier = ShapeShiftingColumnarIntsSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), byteOrder); + columnarInts = supplier.get(); + } + + + private void seedAndEncodeData(final int totalSize) throws IOException + { + int intChunkSize = 1 << (blockSize.getLogBlockSize() - 2); + + vals = new int[totalSize]; + Random rand = new Random(0); + boolean isZero = false; + boolean isConstant = false; + int constant = 0; + + for (int i = 0; i < vals.length; ++i) { + // occasionally write a zero or constant block for funsies + if (i != 0 && (i % intChunkSize == 0)) { + isZero = false; + isConstant = false; + int rando = rand.nextInt(3); + switch (rando) { + case 0: + isZero = true; + break; + case 1: + isConstant = true; + constant = rand.nextInt((1 << 31) - 1); + break; + + } + } + + if (isZero) { + vals[i] = 0; + } else if (isConstant) { + vals[i] = constant; + } else { + vals[i] = rand.nextInt((1 << 31) - 1); + } + } + + serializeAndGetSupplier(); + } + + + // todo: copy pasta, maybe should share these validation methods between tests + private void assertIndexMatchesVals() + { + Assert.assertEquals(vals.length, columnarInts.size()); + + // sequential access + int[] indices = new int[vals.length]; + for (int i = 0, size = columnarInts.size(); i < size; i++) { + Assert.assertEquals("row mismatch at " + i, vals[i], columnarInts.get(i), 0.0); + indices[i] = i; + } + + // random access, limited to 1000 elements for large lists (every element would take too long) + IntArrays.shuffle(indices, ThreadLocalRandom.current()); + final int limit = Math.min(columnarInts.size(), 1000); + for (int i = 0; i < limit; ++i) { + int k = indices[i]; + Assert.assertEquals(vals[k], columnarInts.get(k), 0.0); + } + } +}