From c4e8440c229535d9f6b08128cae4102cbfd30ee9 Mon Sep 17 00:00:00 2001 From: Dave Li Date: Tue, 30 Aug 2016 19:17:46 -0400 Subject: [PATCH] Adds long compression methods (#3148) * add read * update deprecated guava calls * add write and vsizeserde * add benchmark * separate encoding and compression * add header and reformat * update doc * address PR comment * fix buffer order * generate benchmark files * separate encoding strategy and format * fix benchmark * modify supplier write to channel * add float NONE handling * address PR comment * address PR comment 2 --- NOTICE | 6 + .../benchmark/FloatCompressionBenchmark.java | 105 +++ ...loatCompressionBenchmarkFileGenerator.java | 195 ++++++ .../benchmark/LongCompressionBenchmark.java | 108 +++ ...LongCompressionBenchmarkFileGenerator.java | 189 ++++++ .../druid/benchmark/VSizeSerdeBenchmark.java | 212 ++++++ .../extensions-core/kafka-ingestion.md | 7 +- docs/content/ingestion/batch-ingestion.md | 3 +- docs/content/ingestion/tasks.md | 3 +- .../updater/HadoopConverterJobTest.java | 12 +- .../task/HadoopConverterTaskSerDeTest.java | 7 +- .../indexing/common/task/TaskSerdeTest.java | 7 +- .../druid/segment/FloatColumnSerializer.java | 7 +- .../segment/FloatMetricColumnSerializer.java | 18 +- .../main/java/io/druid/segment/IndexIO.java | 6 +- .../java/io/druid/segment/IndexMerger.java | 20 +- .../java/io/druid/segment/IndexMergerV9.java | 23 +- .../main/java/io/druid/segment/IndexSpec.java | 100 ++- .../druid/segment/LongColumnSerializer.java | 18 +- .../segment/LongMetricColumnSerializer.java | 21 +- .../java/io/druid/segment/MetricHolder.java | 32 +- ...> BlockLayoutFloatSupplierSerializer.java} | 103 ++- .../data/BlockLayoutIndexedFloatSupplier.java | 148 ++++ .../data/BlockLayoutIndexedLongSupplier.java | 183 +++++ .../BlockLayoutLongSupplierSerializer.java | 157 +++++ .../data/CompressedFloatsIndexedSupplier.java | 245 +------ .../data/CompressedLongsIndexedSupplier.java | 322 ++------- .../CompressedLongsSupplierSerializer.java | 147 ---- .../data/CompressedObjectStrategy.java | 46 ++ .../segment/data/CompressionFactory.java | 333 +++++++++ .../segment/data/DeltaLongEncodingReader.java | 79 +++ .../segment/data/DeltaLongEncodingWriter.java | 88 +++ .../EntireLayoutFloatSupplierSerializer.java | 122 ++++ .../EntireLayoutIndexedFloatSupplier.java | 89 +++ .../data/EntireLayoutIndexedLongSupplier.java | 87 +++ .../EntireLayoutLongSupplierSerializer.java | 119 ++++ .../segment/data/FloatSupplierSerializer.java | 36 + .../segment/data/GenericIndexedWriter.java | 16 +- .../data/InMemoryCompressedFloats.java | 202 ------ .../segment/data/InMemoryCompressedLongs.java | 213 ------ .../io/druid/segment/data/IndexedLongs.java | 2 - .../IntermediateLongSupplierSerializer.java | 150 +++++ .../segment/data/LongSupplierSerializer.java | 38 ++ .../segment/data/LongsLongEncodingReader.java | 65 ++ .../segment/data/LongsLongEncodingWriter.java | 96 +++ .../segment/data/TableLongEncodingReader.java | 85 +++ .../segment/data/TableLongEncodingWriter.java | 97 +++ .../data/VSizeCompressedObjectStrategy.java | 86 +++ .../io/druid/segment/data/VSizeLongSerde.java | 634 ++++++++++++++++++ .../java/io/druid/segment/IndexIOTest.java | 4 +- .../io/druid/segment/IndexMergerTest.java | 90 +-- .../IndexMergerV9CompatibilityTest.java | 4 +- .../java/io/druid/segment/IndexSpecTest.java | 17 +- .../QueryableIndexIndexableAdapterTest.java | 4 +- .../CompressedFloatsIndexedSupplierTest.java | 351 ---------- .../data/CompressedFloatsSerdeTest.java | 295 ++++++++ ...ompressedFloatsSupplierSerializerTest.java | 136 ---- .../data/CompressedIntsIndexedWriterTest.java | 2 +- .../CompressedLongsIndexedSupplierTest.java | 343 ---------- .../data/CompressedLongsSerdeTest.java | 317 +++++++++ ...CompressedLongsSupplierSerializerTest.java | 126 ---- .../CompressedVSizeIndexedV3WriterTest.java | 2 +- .../CompressedVSizeIntsIndexedWriterTest.java | 2 +- .../segment/data/CompressionStrategyTest.java | 2 +- .../data/InMemoryCompressedFloatsTest.java | 107 --- .../data/InMemoryCompressedLongsTest.java | 108 --- .../segment/data/VSizeLongSerdeTest.java | 151 +++++ .../druid/segment/filter/BaseFilterTest.java | 1 + .../IncrementalIndexAdapterTest.java | 6 +- 69 files changed, 4662 insertions(+), 2493 deletions(-) create mode 100644 benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java create mode 100644 benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java create mode 100644 benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java create mode 100644 benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java create mode 100644 benchmarks/src/main/java/io/druid/benchmark/VSizeSerdeBenchmark.java rename processing/src/main/java/io/druid/segment/data/{CompressedFloatsSupplierSerializer.java => BlockLayoutFloatSupplierSerializer.java} (52%) create mode 100644 processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java create mode 100644 processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java create mode 100644 processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java delete mode 100644 processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressionFactory.java create mode 100644 processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java create mode 100644 processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java create mode 100644 processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedFloatSupplier.java create mode 100644 processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedLongSupplier.java create mode 100644 processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/FloatSupplierSerializer.java delete mode 100644 processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java delete mode 100644 processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java create mode 100644 processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java create mode 100644 processing/src/main/java/io/druid/segment/data/LongsLongEncodingReader.java create mode 100644 processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java create mode 100644 processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java create mode 100644 processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java create mode 100644 processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java create mode 100644 processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java delete mode 100644 processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java create mode 100644 processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java delete mode 100644 processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java delete mode 100644 processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java create mode 100644 processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java delete mode 100644 processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java delete mode 100644 processing/src/test/java/io/druid/segment/data/InMemoryCompressedFloatsTest.java delete mode 100644 processing/src/test/java/io/druid/segment/data/InMemoryCompressedLongsTest.java create mode 100644 processing/src/test/java/io/druid/segment/data/VSizeLongSerdeTest.java diff --git a/NOTICE b/NOTICE index ce265a4e938a..5123303c4312 100644 --- a/NOTICE +++ b/NOTICE @@ -16,3 +16,9 @@ This product contains conjunctive normal form conversion code and a variance agg * https://github.com/apache/hive/blob/branch-2.0/LICENSE (Apache License, Version 2.0) * HOMEPAGE: * https://github.com/apache/hive + +This product contains variable length long deserialization code adapted from Apache Lucene + * LICENSE: + * https://github.com/apache/lucene-solr/blob/master/lucene/LICENSE.txt (Apache License, Version 2.0) + * HOMEPAGE: + * https://github.com/apache/lucene-solr diff --git a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java new file mode 100644 index 000000000000..327de09b6a9e --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java @@ -0,0 +1,105 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.benchmark; + +// Run FloatCompressionBenchmarkFileGenerator to generate the required files before running this benchmark + +import com.google.common.base.Supplier; +import com.google.common.io.Files; +import io.druid.segment.data.CompressedFloatsIndexedSupplier; +import io.druid.segment.data.IndexedFloats; +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.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 25) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class FloatCompressionBenchmark +{ + @Param("floatCompress/") + private static String dirPath; + + @Param({"enumerate", "zipfLow", "zipfHigh", "sequential", "uniform"}) + private static String file; + + @Param({"lz4", "none"}) + private static String strategy; + + private Random rand; + private Supplier supplier; + + @Setup + public void setup() throws Exception + { + File dir = new File(dirPath); + File compFile = new File(dir, file + "-" + strategy); + rand = new Random(); + ByteBuffer buffer = Files.map(compFile); + supplier = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder()); + } + + @Benchmark + public void readContinuous(Blackhole bh) throws IOException + { + IndexedFloats indexedFloats = supplier.get(); + int count = indexedFloats.size(); + float sum = 0; + for (int i = 0; i < count; i++) { + sum += indexedFloats.get(i); + } + bh.consume(sum); + indexedFloats.close(); + } + + @Benchmark + public void readSkipping(Blackhole bh) throws IOException + { + IndexedFloats indexedFloats = supplier.get(); + int count = indexedFloats.size(); + float sum = 0; + for (int i = 0; i < count; i += rand.nextInt(2000)) { + sum += indexedFloats.get(i); + } + bh.consume(sum); + indexedFloats.close(); + } + +} diff --git a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java new file mode 100644 index 000000000000..6029f8716bf8 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java @@ -0,0 +1,195 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.benchmark; + +import com.google.common.collect.ImmutableList; +import com.google.common.io.ByteSink; +import io.druid.benchmark.datagen.BenchmarkColumnSchema; +import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.FloatSupplierSerializer; +import io.druid.segment.data.TmpFileIOPeon; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class FloatCompressionBenchmarkFileGenerator +{ + public static final int ROW_NUM = 5000000; + public static final List compressions = + ImmutableList.of( + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.NONE + ); + + private static String dirPath = "floatCompress/"; + + public static void main(String[] args) throws IOException, URISyntaxException + { + if (args.length >= 1) { + dirPath = args[0]; + } + + BenchmarkColumnSchema enumeratedSchema = BenchmarkColumnSchema.makeEnumerated("", ValueType.FLOAT, true, 1, 0d, + ImmutableList.of( + 0f, + 1.1f, + 2.2f, + 3.3f, + 4.4f + ), + ImmutableList.of( + 0.95, + 0.001, + 0.0189, + 0.03, + 0.0001 + ) + ); + BenchmarkColumnSchema zipfLowSchema = BenchmarkColumnSchema.makeZipf( + "", + ValueType.FLOAT, + true, + 1, + 0d, + -1, + 1000, + 1d + ); + BenchmarkColumnSchema zipfHighSchema = BenchmarkColumnSchema.makeZipf( + "", + ValueType.FLOAT, + true, + 1, + 0d, + -1, + 1000, + 3d + ); + BenchmarkColumnSchema sequentialSchema = BenchmarkColumnSchema.makeSequential( + "", + ValueType.FLOAT, + true, + 1, + 0d, + 1470187671, + 2000000000 + ); + BenchmarkColumnSchema uniformSchema = BenchmarkColumnSchema.makeContinuousUniform( + "", + ValueType.FLOAT, + true, + 1, + 0d, + 0, + 1000 + ); + + Map generators = new HashMap<>(); + generators.put("enumerate", new BenchmarkColumnValueGenerator(enumeratedSchema, 1)); + generators.put("zipfLow", new BenchmarkColumnValueGenerator(zipfLowSchema, 1)); + generators.put("zipfHigh", new BenchmarkColumnValueGenerator(zipfHighSchema, 1)); + generators.put("sequential", new BenchmarkColumnValueGenerator(sequentialSchema, 1)); + generators.put("uniform", new BenchmarkColumnValueGenerator(uniformSchema, 1)); + + File dir = new File(dirPath); + dir.mkdir(); + + // create data files using BenchmarkColunValueGenerator + for (Map.Entry entry : generators.entrySet()) { + final File dataFile = new File(dir, entry.getKey()); + dataFile.delete(); + try (Writer writer = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(dataFile)))) { + for (int i = 0; i < ROW_NUM; i++) { + writer.write((Float) entry.getValue().generateRowValue() + "\n"); + } + } + } + + // create compressed files using all combinations of CompressionStrategy and FloatEncoding provided + for (Map.Entry entry : generators.entrySet()) { + for (CompressedObjectStrategy.CompressionStrategy compression : compressions) { + String name = entry.getKey() + "-" + compression.toString(); + System.out.print(name + ": "); + File compFile = new File(dir, name); + compFile.delete(); + File dataFile = new File(dir, entry.getKey()); + + TmpFileIOPeon iopeon = new TmpFileIOPeon(true); + FloatSupplierSerializer writer = CompressionFactory.getFloatSerializer( + iopeon, + "float", + ByteOrder.nativeOrder(), + compression + ); + BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(dataFile))); + + try (FileChannel output = FileChannel.open( + compFile.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.WRITE + )) { + writer.open(); + String line; + while ((line = br.readLine()) != null) { + writer.add(Float.parseFloat(line)); + } + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + writer.closeAndConsolidate( + new ByteSink() + { + @Override + public OutputStream openStream() throws IOException + { + return baos; + } + } + ); + output.write(ByteBuffer.wrap(baos.toByteArray())); + } + finally { + iopeon.cleanup(); + br.close(); + } + System.out.print(compFile.length() / 1024 + "\n"); + } + } + } +} diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java new file mode 100644 index 000000000000..cfb40ceabd5f --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java @@ -0,0 +1,108 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.benchmark; + +import com.google.common.base.Supplier; +import com.google.common.io.Files; +import io.druid.segment.data.CompressedLongsIndexedSupplier; +import io.druid.segment.data.IndexedLongs; +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.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Random; +import java.util.concurrent.TimeUnit; + +// Run LongCompressionBenchmarkFileGenerator to generate the required files before running this benchmark + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 25) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class LongCompressionBenchmark +{ + @Param("longCompress/") + private static String dirPath; + + @Param({"enumerate", "zipfLow", "zipfHigh", "sequential", "uniform"}) + private static String file; + + @Param({"auto", "longs"}) + private static String format; + + @Param({"lz4", "none"}) + private static String strategy; + + private Random rand; + private Supplier supplier; + + @Setup + public void setup() throws Exception + { + File dir = new File(dirPath); + File compFile = new File(dir, file + "-" + strategy + "-" + format); + rand = new Random(); + ByteBuffer buffer = Files.map(compFile); + supplier = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder()); + } + + @Benchmark + public void readContinuous(Blackhole bh) throws IOException + { + IndexedLongs indexedLongs = supplier.get(); + int count = indexedLongs.size(); + long sum = 0; + for (int i = 0; i < count; i++) { + sum += indexedLongs.get(i); + } + bh.consume(sum); + indexedLongs.close(); + } + + @Benchmark + public void readSkipping(Blackhole bh) throws IOException + { + IndexedLongs indexedLongs = supplier.get(); + int count = indexedLongs.size(); + long sum = 0; + for (int i = 0; i < count; i += rand.nextInt(2000)) { + sum += indexedLongs.get(i); + } + bh.consume(sum); + indexedLongs.close(); + } + +} + diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java new file mode 100644 index 000000000000..14a94bf7c310 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java @@ -0,0 +1,189 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.benchmark; + +import com.google.common.collect.ImmutableList; +import com.google.common.io.ByteSink; +import io.druid.benchmark.datagen.BenchmarkColumnSchema; +import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator; +import io.druid.segment.column.ValueType; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.LongSupplierSerializer; +import io.druid.segment.data.TmpFileIOPeon; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class LongCompressionBenchmarkFileGenerator +{ + public static final int ROW_NUM = 5000000; + public static final List compressions = + ImmutableList.of(CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.NONE); + public static final List encodings = + ImmutableList.of(CompressionFactory.LongEncodingStrategy.AUTO, CompressionFactory.LongEncodingStrategy.LONGS); + + private static String dirPath = "longCompress/"; + + public static void main(String[] args) throws IOException, URISyntaxException + { + if (args.length >= 1) { + dirPath = args[0]; + } + + BenchmarkColumnSchema enumeratedSchema = BenchmarkColumnSchema.makeEnumerated("", ValueType.LONG, true, 1, 0d, + ImmutableList.of( + 0, + 1, + 2, + 3, + 4 + ), + ImmutableList.of( + 0.95, + 0.001, + 0.0189, + 0.03, + 0.0001 + ) + ); + BenchmarkColumnSchema zipfLowSchema = BenchmarkColumnSchema.makeZipf("", ValueType.LONG, true, 1, 0d, -1, 1000, 1d); + BenchmarkColumnSchema zipfHighSchema = BenchmarkColumnSchema.makeZipf( + "", + ValueType.LONG, + true, + 1, + 0d, + -1, + 1000, + 3d + ); + BenchmarkColumnSchema sequentialSchema = BenchmarkColumnSchema.makeSequential( + "", + ValueType.LONG, + true, + 1, + 0d, + 1470187671, + 2000000000 + ); + BenchmarkColumnSchema uniformSchema = BenchmarkColumnSchema.makeDiscreteUniform( + "", + ValueType.LONG, + true, + 1, + 0d, + 0, + 1000 + ); + + Map generators = new HashMap<>(); + generators.put("enumerate", new BenchmarkColumnValueGenerator(enumeratedSchema, 1)); + generators.put("zipfLow", new BenchmarkColumnValueGenerator(zipfLowSchema, 1)); + generators.put("zipfHigh", new BenchmarkColumnValueGenerator(zipfHighSchema, 1)); + generators.put("sequential", new BenchmarkColumnValueGenerator(sequentialSchema, 1)); + generators.put("uniform", new BenchmarkColumnValueGenerator(uniformSchema, 1)); + + File dir = new File(dirPath); + dir.mkdir(); + + // create data files using BenchmarkColunValueGenerator + for (Map.Entry entry : generators.entrySet()) { + final File dataFile = new File(dir, entry.getKey()); + dataFile.delete(); + try (Writer writer = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(dataFile)))) { + for (int i = 0; i < ROW_NUM; i++) { + writer.write((long) entry.getValue().generateRowValue() + "\n"); + } + } + } + + // create compressed files using all combinations of CompressionStrategy and LongEncoding provided + for (Map.Entry entry : generators.entrySet()) { + for (CompressedObjectStrategy.CompressionStrategy compression : compressions) { + for (CompressionFactory.LongEncodingStrategy encoding : encodings) { + String name = entry.getKey() + "-" + compression.toString() + "-" + encoding.toString(); + System.out.print(name + ": "); + File compFile = new File(dir, name); + compFile.delete(); + File dataFile = new File(dir, entry.getKey()); + + TmpFileIOPeon iopeon = new TmpFileIOPeon(true); + LongSupplierSerializer writer = CompressionFactory.getLongSerializer( + iopeon, + "long", + ByteOrder.nativeOrder(), + encoding, + compression + ); + BufferedReader br = new BufferedReader(new InputStreamReader(new FileInputStream(dataFile))); + + try (FileChannel output = FileChannel.open( + compFile.toPath(), + StandardOpenOption.CREATE_NEW, + StandardOpenOption.WRITE + )) { + writer.open(); + String line; + while ((line = br.readLine()) != null) { + writer.add(Long.parseLong(line)); + } + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + writer.closeAndConsolidate( + new ByteSink() + { + @Override + public OutputStream openStream() throws IOException + { + return baos; + } + } + ); + output.write(ByteBuffer.wrap(baos.toByteArray())); + } + finally { + iopeon.cleanup(); + br.close(); + } + System.out.print(compFile.length() / 1024 + "\n"); + } + } + } + } +} diff --git a/benchmarks/src/main/java/io/druid/benchmark/VSizeSerdeBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/VSizeSerdeBenchmark.java new file mode 100644 index 000000000000..490128cc5f60 --- /dev/null +++ b/benchmarks/src/main/java/io/druid/benchmark/VSizeSerdeBenchmark.java @@ -0,0 +1,212 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.benchmark; + +import com.google.common.io.Files; +import io.druid.segment.data.VSizeLongSerde; +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 java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.Writer; +import java.net.URISyntaxException; +import java.nio.ByteBuffer; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 10) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class VSizeSerdeBenchmark +{ + @Param({"500000"}) + private int values; + + private VSizeLongSerde.LongDeserializer d1; + private VSizeLongSerde.LongDeserializer d2; + private VSizeLongSerde.LongDeserializer d4; + private VSizeLongSerde.LongDeserializer d8; + private VSizeLongSerde.LongDeserializer d12; + private VSizeLongSerde.LongDeserializer d16; + private VSizeLongSerde.LongDeserializer d20; + private VSizeLongSerde.LongDeserializer d24; + private VSizeLongSerde.LongDeserializer d32; + private VSizeLongSerde.LongDeserializer d40; + private VSizeLongSerde.LongDeserializer d48; + private VSizeLongSerde.LongDeserializer d56; + private VSizeLongSerde.LongDeserializer d64; + private long sum; + private File dummy; + + @Setup + public void setup() throws IOException, URISyntaxException + { + // this uses a dummy file of sufficient size to construct a mappedByteBuffer instead of using ByteBuffer.allocate + // to construct a heapByteBuffer since they have different performance + File base = new File(this.getClass().getClassLoader().getResource("").toURI()); + dummy = new File(base, "dummy"); + try (Writer writer = new BufferedWriter(new FileWriter(dummy))) { + String EMPTY_STRING = " "; + for (int i = 0; i < values + 10; i++) { + writer.write(EMPTY_STRING); + } + } + ByteBuffer buffer = Files.map(dummy); + d1 = VSizeLongSerde.getDeserializer(1, buffer, 10); + d2 = VSizeLongSerde.getDeserializer(2, buffer, 10); + d4 = VSizeLongSerde.getDeserializer(4, buffer, 10); + d8 = VSizeLongSerde.getDeserializer(8, buffer, 10); + d12 = VSizeLongSerde.getDeserializer(12, buffer, 10); + d16 = VSizeLongSerde.getDeserializer(16, buffer, 10); + d20 = VSizeLongSerde.getDeserializer(20, buffer, 10); + d24 = VSizeLongSerde.getDeserializer(24, buffer, 10); + d32 = VSizeLongSerde.getDeserializer(32, buffer, 10); + d40 = VSizeLongSerde.getDeserializer(40, buffer, 10); + d48 = VSizeLongSerde.getDeserializer(48, buffer, 10); + d56 = VSizeLongSerde.getDeserializer(56, buffer, 10); + d64 = VSizeLongSerde.getDeserializer(64, buffer, 10); + } + + @TearDown + public void tearDown() + { + dummy.delete(); + System.out.println(sum); + } + + @Benchmark + public void read1() + { + for (int i = 0; i < values; i++) { + sum += d1.get(i); + } + } + + @Benchmark + public void read2() + { + for (int i = 0; i < values; i++) { + sum += d2.get(i); + } + } + + @Benchmark + public void read4() + { + for (int i = 0; i < values; i++) { + sum += d4.get(i); + } + } + + @Benchmark + public void read8() + { + for (int i = 0; i < values; i++) { + sum += d8.get(i); + } + } + + @Benchmark + public void readd12() + { + for (int i = 0; i < values; i++) { + sum += d12.get(i); + } + } + + @Benchmark + public void readd16() + { + for (int i = 0; i < values; i++) { + sum += d16.get(i); + } + } + + @Benchmark + public void readd20() + { + for (int i = 0; i < values; i++) { + sum += d20.get(i); + } + } + + @Benchmark + public void readd24() + { + for (int i = 0; i < values; i++) { + sum += d24.get(i); + } + } + + @Benchmark + public void readd32() + { + for (int i = 0; i < values; i++) { + sum += d32.get(i); + } + } + + @Benchmark + public void readd40() + { + for (int i = 0; i < values; i++) { + sum += d40.get(i); + } + } + + @Benchmark + public void readd48() + { + for (int i = 0; i < values; i++) { + sum += d48.get(i); + } + } + + @Benchmark + public void readd56() + { + for (int i = 0; i < values; i++) { + sum += d56.get(i); + } + } + + @Benchmark + public void readd64() + { + for (int i = 0; i < values; i++) { + sum += d64.get(i); + } + } +} diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index cea6d56c1458..84ba9243456d 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -128,9 +128,10 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |Field|Type|Description|Required| |-----|----|-----------|--------| -|`bitmap`|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| -|`dimensionCompression`|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| -|`metricCompression`|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| +|dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| +|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| ##### Bitmap types diff --git a/docs/content/ingestion/batch-ingestion.md b/docs/content/ingestion/batch-ingestion.md index 874c2af75e4b..cd25908aa364 100644 --- a/docs/content/ingestion/batch-ingestion.md +++ b/docs/content/ingestion/batch-ingestion.md @@ -190,7 +190,8 @@ for more details. |-----|----|-----------|--------| |bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| |dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| -|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| +|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| ##### Bitmap types diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index ead17089335a..49e368b12e68 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -123,7 +123,8 @@ compression formats. The indexSpec is optional and default parameters will be us |-----|----|-----------|--------| |bitmap|Object|Compression format for bitmap indexes. Should be a JSON object; see below for options.|no (defaults to Concise)| |dimensionCompression|String|Compression format for dimension columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| -|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, or `uncompressed`.|no (default == `LZ4`)| +|metricCompression|String|Compression format for metric columns. Choose from `LZ4`, `LZF`, `uncompressed`, or `none`.|no (default == `LZ4`)| +|longEncoding|String|Encoding format for metric and dimension columns with type long. Choose from `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as is with 8 bytes each.|no (default == `longs`)| ##### Bitmap types diff --git a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java index 9cc9da7a1438..51bcb3303cfd 100644 --- a/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java +++ b/indexing-hadoop/src/test/java/io/druid/indexer/updater/HadoopConverterJobTest.java @@ -56,6 +56,8 @@ import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.segment.IndexSpec; import io.druid.segment.TestIndex; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.RoaringBitmapSerdeFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -284,7 +286,10 @@ public MetadataSegmentManagerConfig get() new HadoopDruidConverterConfig( DATASOURCE, interval, - new IndexSpec(new RoaringBitmapSerdeFactory(null), "uncompressed", "uncompressed"), + new IndexSpec(new RoaringBitmapSerdeFactory(null), + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressionFactory.LongEncodingStrategy.LONGS), oldSemgments, true, tmpDir.toURI(), @@ -387,7 +392,10 @@ public MetadataSegmentManagerConfig get() new HadoopDruidConverterConfig( DATASOURCE, interval, - new IndexSpec(new RoaringBitmapSerdeFactory(null), "uncompressed", "uncompressed"), + new IndexSpec(new RoaringBitmapSerdeFactory(null), + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressionFactory.LongEncodingStrategy.LONGS), oldSemgments, true, tmpDir.toURI(), diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/HadoopConverterTaskSerDeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/HadoopConverterTaskSerDeTest.java index 58cbc01eb8e2..c120845eb1b0 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/HadoopConverterTaskSerDeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/HadoopConverterTaskSerDeTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableMap; import io.druid.indexing.common.TestUtils; import io.druid.segment.IndexSpec; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -50,7 +52,10 @@ public class HadoopConverterTaskSerDeTest private static final ShardSpec SHARD_SPEC = NoneShardSpec.instance(); private static final int BINARY_VERSION = 34718; private static final long SEGMENT_SIZE = 7483901348790L; - private static final IndexSpec INDEX_SPEC = new IndexSpec(new ConciseBitmapSerdeFactory(), "lz4", "lzf"); + private static final IndexSpec INDEX_SPEC = new IndexSpec(new ConciseBitmapSerdeFactory(), + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZF, + CompressionFactory.LongEncodingStrategy.LONGS); private static final DataSegment DATA_SEGMENT = new DataSegment( DATA_SOURCE, INTERVAL, diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index ab06fa4dcf57..9889c9f4569d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java @@ -36,6 +36,8 @@ import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.segment.IndexSpec; +import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.RoaringBitmapSerdeFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; @@ -518,7 +520,10 @@ public void testSegmentConvertSerde() throws IOException ); final ConvertSegmentTask convertSegmentTaskOriginal = ConvertSegmentTask.create( segment, - new IndexSpec(new RoaringBitmapSerdeFactory(null), "lzf", "uncompressed"), + new IndexSpec(new RoaringBitmapSerdeFactory(null), + CompressedObjectStrategy.CompressionStrategy.LZF, + CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressionFactory.LongEncodingStrategy.LONGS), false, true, null diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java index 1cb828262694..6031d02da518 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java @@ -19,8 +19,9 @@ package io.druid.segment; -import io.druid.segment.data.CompressedFloatsSupplierSerializer; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.FloatSupplierSerializer; import io.druid.segment.data.IOPeon; import java.io.IOException; @@ -42,7 +43,7 @@ public static FloatColumnSerializer create( private final String filenameBase; private final ByteOrder byteOrder; private final CompressedObjectStrategy.CompressionStrategy compression; - private CompressedFloatsSupplierSerializer writer; + private FloatSupplierSerializer writer; public FloatColumnSerializer( IOPeon ioPeon, @@ -60,7 +61,7 @@ public FloatColumnSerializer( @Override public void open() throws IOException { - writer = CompressedFloatsSupplierSerializer.create( + writer = CompressionFactory.getFloatSerializer( ioPeon, String.format("%s.float_column", filenameBase), byteOrder, diff --git a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java index c7ed37a03880..43bd4dd6a01c 100644 --- a/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatMetricColumnSerializer.java @@ -19,9 +19,11 @@ package io.druid.segment; +import com.google.common.io.FileWriteMode; import com.google.common.io.Files; -import io.druid.segment.data.CompressedFloatsSupplierSerializer; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.FloatSupplierSerializer; import io.druid.segment.data.IOPeon; import java.io.File; @@ -34,26 +36,28 @@ public class FloatMetricColumnSerializer implements MetricColumnSerializer private final String metricName; private final IOPeon ioPeon; private final File outDir; + private final CompressedObjectStrategy.CompressionStrategy compression; - private CompressedFloatsSupplierSerializer writer; + private FloatSupplierSerializer writer; public FloatMetricColumnSerializer( String metricName, File outDir, - IOPeon ioPeon + IOPeon ioPeon, + CompressedObjectStrategy.CompressionStrategy compression ) { this.metricName = metricName; this.ioPeon = ioPeon; this.outDir = outDir; + this.compression = compression; } @Override public void open() throws IOException { - writer = CompressedFloatsSupplierSerializer.create( - ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, - CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + writer = CompressionFactory.getFloatSerializer( + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, compression ); writer.open(); @@ -72,7 +76,7 @@ public void close() throws IOException final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); outFile.delete(); MetricHolder.writeFloatMetric( - Files.newOutputStreamSupplier(outFile, true), metricName, writer + Files.asByteSink(outFile, FileWriteMode.APPEND), metricName, writer ); IndexIO.checkFileSize(outFile); diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index e90fbdb49524..73f1ac8aa999 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -693,7 +693,7 @@ public int size() builder.setHasMultipleValues(true); } - final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompressionStrategy(); + final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); final DictionaryEncodedColumnPartSerde.LegacySerializerBuilder columnPartBuilder = DictionaryEncodedColumnPartSerde .legacySerializerBuilder() @@ -704,7 +704,7 @@ public int size() .withByteOrder(BYTE_ORDER); if (singleValCol != null) { - if (compressionStrategy != null) { + if (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) { columnPartBuilder.withSingleValuedColumn( CompressedVSizeIntsIndexedSupplier.fromList( singleValCol, @@ -717,7 +717,7 @@ public int size() } else { columnPartBuilder.withSingleValuedColumn(VSizeIndexedInts.fromList(singleValCol, dictionary.size())); } - } else if (compressionStrategy != null) { + } else if (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) { columnPartBuilder.withMultiValuedColumn( CompressedVSizeIndexedSupplier.fromIterable( multiValCol, diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 1bace7920d02..10622015749f 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -33,8 +33,10 @@ import com.google.common.collect.Ordering; import com.google.common.collect.PeekingIterator; import com.google.common.collect.Sets; +import com.google.common.io.ByteSink; import com.google.common.io.ByteStreams; import com.google.common.io.Closer; +import com.google.common.io.FileWriteMode; import com.google.common.io.Files; import com.google.common.io.OutputSupplier; import com.google.common.primitives.Ints; @@ -65,8 +67,9 @@ import io.druid.segment.column.ValueType; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferWriter; -import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.IOPeon; @@ -788,8 +791,9 @@ public void close() throws IOException rowMergerFn ); - CompressedLongsSupplierSerializer timeWriter = CompressedLongsSupplierSerializer.create( - ioPeon, "little_end_time", IndexIO.BYTE_ORDER, CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + LongSupplierSerializer timeWriter = CompressionFactory.getLongSerializer( + ioPeon, "little_end_time", IndexIO.BYTE_ORDER, indexSpec.getLongEncoding(), + CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY ); timeWriter.open(); @@ -802,14 +806,16 @@ public void close() throws IOException } ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); + final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); + final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); for (String metric : mergedMetrics) { ValueType type = valueTypes.get(metric); switch (type) { case LONG: - metWriters.add(new LongMetricColumnSerializer(metric, v8OutDir, ioPeon)); + metWriters.add(new LongMetricColumnSerializer(metric, v8OutDir, ioPeon, metCompression, longEncoding)); break; case FLOAT: - metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon)); + metWriters.add(new FloatMetricColumnSerializer(metric, v8OutDir, ioPeon, metCompression)); break; case COMPLEX: final String typeName = metricTypeNames.get(metric); @@ -889,7 +895,7 @@ public void close() throws IOException final File timeFile = IndexIO.makeTimeFile(v8OutDir, IndexIO.BYTE_ORDER); timeFile.delete(); - OutputSupplier out = Files.newOutputStreamSupplier(timeFile, true); + ByteSink out = Files.asByteSink(timeFile, FileWriteMode.APPEND); timeWriter.closeAndConsolidate(out); IndexIO.checkFileSize(timeFile); @@ -915,7 +921,7 @@ public void close() throws IOException final File invertedFile = new File(v8OutDir, "inverted.drd"); Files.touch(invertedFile); - out = Files.newOutputStreamSupplier(invertedFile, true); + out = Files.asByteSink(invertedFile, FileWriteMode.APPEND); final File geoFile = new File(v8OutDir, "spatial.drd"); Files.touch(geoFile); diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index ae555ad967ff..6936f4864066 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -54,6 +54,7 @@ import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressedVSizeIndexedV3Writer; import io.druid.segment.data.CompressedVSizeIntsIndexedWriter; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.IOPeon; @@ -207,7 +208,7 @@ public void close() throws IOException convertMissingDimsFlags, rowMergerFn ); - final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon); + final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon, indexSpec); final ArrayList dimWriters = setupDimensionWriters( ioPeon, mergedDimensions, dimCapabilities, dimCardinalities, indexSpec ); @@ -356,7 +357,7 @@ private void makeDimensionColumns( long startTime = System.currentTimeMillis(); final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); - final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompressionStrategy(); + final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); for (int i = 0; i < mergedDimensions.size(); ++i) { long dimStartTime = System.currentTimeMillis(); final String dim = mergedDimensions.get(i); @@ -381,7 +382,7 @@ private void makeDimensionColumns( final DictionaryEncodedColumnPartSerde.SerializerBuilder partBuilder = DictionaryEncodedColumnPartSerde .serializerBuilder() .withDictionary(dimValueWriters.get(i)) - .withValue(dimWriters.get(i), hasMultiValue, compressionStrategy != null) + .withValue(dimWriters.get(i), hasMultiValue, compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) .withBitmapSerdeFactory(bitmapSerdeFactory) .withBitmapIndex(bitmapIndexWriters.get(i)) .withSpatialIndex(spatialIndexWriters.get(i)) @@ -723,10 +724,11 @@ private void mergeIndexesAndWriteColumns( progress.stopSection(section); } - private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon) throws IOException + private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon, final IndexSpec indexSpec) throws IOException { LongColumnSerializer timeWriter = LongColumnSerializer.create( - ioPeon, "little_end_time", CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + ioPeon, "little_end_time", CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY, + indexSpec.getLongEncoding() ); // we will close this writer after we added all the timestamps timeWriter.open(); @@ -742,13 +744,14 @@ private ArrayList setupMetricsWriters( ) throws IOException { ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompressionStrategy(); + final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); + final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); for (String metric : mergedMetrics) { ValueType type = metricsValueTypes.get(metric); GenericColumnSerializer writer; switch (type) { case LONG: - writer = LongColumnSerializer.create(ioPeon, metric, metCompression); + writer = LongColumnSerializer.create(ioPeon, metric, metCompression, longEncoding); break; case FLOAT: writer = FloatColumnSerializer.create(ioPeon, metric, metCompression); @@ -780,7 +783,7 @@ private ArrayList setupDimensionWriters( ) throws IOException { ArrayList dimWriters = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final CompressedObjectStrategy.CompressionStrategy dimCompression = indexSpec.getDimensionCompressionStrategy(); + final CompressedObjectStrategy.CompressionStrategy dimCompression = indexSpec.getDimensionCompression(); for (int dimIndex = 0; dimIndex < mergedDimensions.size(); ++dimIndex) { String dim = mergedDimensions.get(dimIndex); int cardinality = dimCardinalities.get(dim); @@ -788,11 +791,11 @@ private ArrayList setupDimensionWriters( String filenameBase = String.format("%s.forward_dim", dim); IndexedIntsWriter writer; if (capabilities.hasMultipleValues()) { - writer = (dimCompression != null) + writer = (dimCompression != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) ? CompressedVSizeIndexedV3Writer.create(ioPeon, filenameBase, cardinality, dimCompression) : new VSizeIndexedWriter(ioPeon, filenameBase, cardinality); } else { - writer = (dimCompression != null) + writer = (dimCompression != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) ? CompressedVSizeIntsIndexedWriter.create(ioPeon, filenameBase, cardinality, dimCompression) : new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality); } diff --git a/processing/src/main/java/io/druid/segment/IndexSpec.java b/processing/src/main/java/io/druid/segment/IndexSpec.java index 7674dc971854..8bb8cc15020f 100644 --- a/processing/src/main/java/io/druid/segment/IndexSpec.java +++ b/processing/src/main/java/io/druid/segment/IndexSpec.java @@ -21,15 +21,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; import com.google.common.collect.Sets; +import io.druid.segment.data.BitmapSerde; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; -import javax.annotation.Nullable; import java.util.Arrays; import java.util.Set; @@ -41,28 +40,26 @@ */ public class IndexSpec { - public static final String UNCOMPRESSED = "uncompressed"; - public static final String DEFAULT_METRIC_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY.name().toLowerCase(); - public static final String DEFAULT_DIMENSION_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY.name().toLowerCase(); - - private static final Set COMPRESSION_NAMES = Sets.newHashSet( - Iterables.transform( - Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values()), - new Function() - { - @Nullable - @Override - public String apply(CompressedObjectStrategy.CompressionStrategy input) - { - return input.name().toLowerCase(); - } - } - ) + public static final CompressedObjectStrategy.CompressionStrategy DEFAULT_METRIC_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; + public static final CompressedObjectStrategy.CompressionStrategy DEFAULT_DIMENSION_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; + public static final CompressionFactory.LongEncodingStrategy DEFAULT_LONG_ENCODING = CompressionFactory.DEFAULT_LONG_ENCODING_STRATEGY; + + private static final Set METRIC_COMPRESSION = Sets.newHashSet( + Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values()) + ); + + private static final Set DIMENSION_COMPRESSION = Sets.newHashSet( + Arrays.asList(CompressedObjectStrategy.CompressionStrategy.noNoneValues()) + ); + + private static final Set LONG_ENCODING_NAMES = Sets.newHashSet( + Arrays.asList(CompressionFactory.LongEncodingStrategy.values()) ); private final BitmapSerdeFactory bitmapSerdeFactory; - private final String dimensionCompression; - private final String metricCompression; + private final CompressedObjectStrategy.CompressionStrategy dimensionCompression; + private final CompressedObjectStrategy.CompressionStrategy metricCompression; + private final CompressionFactory.LongEncodingStrategy longEncoding; /** @@ -70,7 +67,7 @@ public String apply(CompressedObjectStrategy.CompressionStrategy input) */ public IndexSpec() { - this(null, null, null); + this(null, null, null, null); } /** @@ -79,30 +76,38 @@ public IndexSpec() * * @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 @{link BitmapSerde.DefaultBitmapSerdeFactory} + * setting, or, if none was set, uses the default {@link BitmapSerde.DefaultBitmapSerdeFactory} * - * @param dimensionCompression compression format for dimension columns, null to use the default - * Defaults to @{link CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY} + * @param dimensionCompression compression format for dimension columns, null to use the default. + * Defaults to {@link CompressedObjectStrategy#DEFAULT_COMPRESSION_STRATEGY} * * @param metricCompression compression format for metric columns, null to use the default. - * Defaults to @{link CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY} + * Defaults to {@link CompressedObjectStrategy#DEFAULT_COMPRESSION_STRATEGY} + * + * @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} */ @JsonCreator public IndexSpec( @JsonProperty("bitmap") BitmapSerdeFactory bitmapSerdeFactory, - @JsonProperty("dimensionCompression") String dimensionCompression, - @JsonProperty("metricCompression") String metricCompression + @JsonProperty("dimensionCompression") CompressedObjectStrategy.CompressionStrategy dimensionCompression, + @JsonProperty("metricCompression") CompressedObjectStrategy.CompressionStrategy metricCompression, + @JsonProperty("longEncoding") CompressionFactory.LongEncodingStrategy longEncoding ) { - Preconditions.checkArgument(dimensionCompression == null || dimensionCompression.equals(UNCOMPRESSED) || COMPRESSION_NAMES.contains(dimensionCompression), + Preconditions.checkArgument(dimensionCompression == null || DIMENSION_COMPRESSION.contains(dimensionCompression), "Unknown compression type[%s]", dimensionCompression); - Preconditions.checkArgument(metricCompression == null || COMPRESSION_NAMES.contains(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); + this.bitmapSerdeFactory = bitmapSerdeFactory != null ? bitmapSerdeFactory : new ConciseBitmapSerdeFactory(); - this.metricCompression = metricCompression; - this.dimensionCompression = dimensionCompression; + this.dimensionCompression = dimensionCompression == null ?DEFAULT_DIMENSION_COMPRESSION : dimensionCompression; + this.metricCompression = metricCompression == null ? DEFAULT_METRIC_COMPRESSION : metricCompression; + this.longEncoding = longEncoding == null ? DEFAULT_LONG_ENCODING : longEncoding; } @JsonProperty("bitmap") @@ -111,36 +116,22 @@ public BitmapSerdeFactory getBitmapSerdeFactory() return bitmapSerdeFactory; } - @JsonProperty("dimensionCompression") - public String getDimensionCompression() + @JsonProperty + public CompressedObjectStrategy.CompressionStrategy getDimensionCompression() { return dimensionCompression; } - @JsonProperty("metricCompression") - public String getMetricCompression() + @JsonProperty + public CompressedObjectStrategy.CompressionStrategy getMetricCompression() { return metricCompression; } - public CompressedObjectStrategy.CompressionStrategy getMetricCompressionStrategy() + @JsonProperty + public CompressionFactory.LongEncodingStrategy getLongEncoding() { - return CompressedObjectStrategy.CompressionStrategy.valueOf( - (metricCompression == null ? DEFAULT_METRIC_COMPRESSION : metricCompression).toUpperCase() - ); - } - - public CompressedObjectStrategy.CompressionStrategy getDimensionCompressionStrategy() - { - return dimensionCompression == null ? - dimensionCompressionStrategyForName(DEFAULT_DIMENSION_COMPRESSION) : - dimensionCompressionStrategyForName(dimensionCompression); - } - - private static CompressedObjectStrategy.CompressionStrategy dimensionCompressionStrategyForName(String compression) - { - return compression.equals(UNCOMPRESSED) ? null : - CompressedObjectStrategy.CompressionStrategy.valueOf(compression.toUpperCase()); + return longEncoding; } @Override @@ -168,7 +159,6 @@ public boolean equals(Object o) return !(metricCompression != null ? !metricCompression.equals(indexSpec.metricCompression) : indexSpec.metricCompression != null); - } @Override diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java index fa03c86d1251..80e803af2fa0 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java @@ -19,8 +19,9 @@ package io.druid.segment; -import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.IOPeon; import java.io.IOException; @@ -32,38 +33,43 @@ public class LongColumnSerializer implements GenericColumnSerializer public static LongColumnSerializer create( IOPeon ioPeon, String filenameBase, - CompressedObjectStrategy.CompressionStrategy compression + CompressedObjectStrategy.CompressionStrategy compression, + CompressionFactory.LongEncodingStrategy encoding ) { - return new LongColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression); + return new LongColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression, encoding); } private final IOPeon ioPeon; private final String filenameBase; private final ByteOrder byteOrder; private final CompressedObjectStrategy.CompressionStrategy compression; - private CompressedLongsSupplierSerializer writer; + private final CompressionFactory.LongEncodingStrategy encoding; + private LongSupplierSerializer writer; public LongColumnSerializer( IOPeon ioPeon, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + CompressedObjectStrategy.CompressionStrategy compression, + CompressionFactory.LongEncodingStrategy encoding ) { this.ioPeon = ioPeon; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; + this.encoding = encoding; } @Override public void open() throws IOException { - writer = CompressedLongsSupplierSerializer.create( + writer = CompressionFactory.getLongSerializer( ioPeon, String.format("%s.long_column", filenameBase), byteOrder, + encoding, compression ); writer.open(); diff --git a/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java index acaf20b02249..24e71f6c3d8a 100644 --- a/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/LongMetricColumnSerializer.java @@ -19,10 +19,12 @@ package io.druid.segment; +import com.google.common.io.FileWriteMode; import com.google.common.io.Files; -import io.druid.segment.data.CompressedLongsSupplierSerializer; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.IOPeon; +import io.druid.segment.data.LongSupplierSerializer; import java.io.File; import java.io.IOException; @@ -34,26 +36,31 @@ public class LongMetricColumnSerializer implements MetricColumnSerializer private final String metricName; private final IOPeon ioPeon; private final File outDir; + private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionFactory.LongEncodingStrategy encoding; - private CompressedLongsSupplierSerializer writer; + private LongSupplierSerializer writer; public LongMetricColumnSerializer( String metricName, File outDir, - IOPeon ioPeon + IOPeon ioPeon, + CompressedObjectStrategy.CompressionStrategy compression, + CompressionFactory.LongEncodingStrategy encoding ) { this.metricName = metricName; this.ioPeon = ioPeon; this.outDir = outDir; + this.compression = compression; + this.encoding = encoding; } @Override public void open() throws IOException { - writer = CompressedLongsSupplierSerializer.create( - ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, - CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY + writer = CompressionFactory.getLongSerializer( + ioPeon, String.format("%s_little", metricName), IndexIO.BYTE_ORDER, encoding, compression ); writer.open(); @@ -72,7 +79,7 @@ public void close() throws IOException final File outFile = IndexIO.makeMetricFile(outDir, metricName, IndexIO.BYTE_ORDER); outFile.delete(); MetricHolder.writeLongMetric( - Files.newOutputStreamSupplier(outFile, true), metricName, writer + Files.asByteSink(outFile, FileWriteMode.APPEND), metricName, writer ); IndexIO.checkFileSize(outFile); diff --git a/processing/src/main/java/io/druid/segment/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java index 61c242c7ca5e..524b3df6a9d4 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.io.ByteSink; import com.google.common.io.ByteStreams; import com.google.common.io.InputSupplier; import com.google.common.io.OutputSupplier; @@ -26,9 +27,9 @@ import com.metamx.common.ISE; import io.druid.common.utils.SerializerUtils; import io.druid.segment.data.CompressedFloatsIndexedSupplier; -import io.druid.segment.data.CompressedFloatsSupplierSerializer; import io.druid.segment.data.CompressedLongsIndexedSupplier; -import io.druid.segment.data.CompressedLongsSupplierSerializer; +import io.druid.segment.data.FloatSupplierSerializer; +import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.Indexed; @@ -83,20 +84,20 @@ public static void writeComplexMetric( } public static void writeFloatMetric( - OutputSupplier outSupplier, String name, CompressedFloatsSupplierSerializer column + ByteSink outSupplier, String name, FloatSupplierSerializer column ) throws IOException { - ByteStreams.write(version, outSupplier); + outSupplier.write(version); serializerUtils.writeString(outSupplier, name); serializerUtils.writeString(outSupplier, "float"); column.closeAndConsolidate(outSupplier); } public static void writeLongMetric( - OutputSupplier outSupplier, String name, CompressedLongsSupplierSerializer column + ByteSink outSupplier, String name, LongSupplierSerializer column ) throws IOException { - ByteStreams.write(version, outSupplier); + outSupplier.write(version); serializerUtils.writeString(outSupplier, name); serializerUtils.writeString(outSupplier, "long"); column.closeAndConsolidate(outSupplier); @@ -231,25 +232,6 @@ public Indexed getComplexType() return complexType; } - public MetricHolder convertByteOrder(ByteOrder order) - { - MetricHolder retVal; - switch (type) { - case LONG: - retVal = new MetricHolder(name, typeName); - retVal.longType = longType.convertByteOrder(order); - return retVal; - case FLOAT: - retVal = new MetricHolder(name, typeName); - retVal.floatType = floatType.convertByteOrder(order); - return retVal; - case COMPLEX: - return this; - } - - return null; - } - private void assertType(MetricType type) { if (this.type != type) { diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java similarity index 52% rename from processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java rename to processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java index 31d045ea4847..c8d47d150763 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java @@ -19,13 +19,17 @@ package io.druid.segment.data; +import com.google.common.io.ByteSink; import com.google.common.io.ByteStreams; -import com.google.common.io.OutputSupplier; +import com.google.common.io.CountingOutputStream; +import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import io.druid.collections.ResourceHolder; import io.druid.collections.StupidResourceHolder; +import io.druid.segment.CompressedPools; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -34,70 +38,50 @@ import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -/** - */ -public class CompressedFloatsSupplierSerializer +public class BlockLayoutFloatSupplierSerializer implements FloatSupplierSerializer { - public static CompressedFloatsSupplierSerializer create( - IOPeon ioPeon, - final String filenameBase, - final ByteOrder order, - final CompressedObjectStrategy.CompressionStrategy compression - ) throws IOException - { - return create(ioPeon, filenameBase, CompressedFloatsIndexedSupplier.MAX_FLOATS_IN_BUFFER, order, compression); - } - - public static CompressedFloatsSupplierSerializer create( - IOPeon ioPeon, - final String filenameBase, - final int sizePer, - final ByteOrder order, - final CompressedObjectStrategy.CompressionStrategy compression - ) throws IOException - { - final CompressedFloatsSupplierSerializer retVal = new CompressedFloatsSupplierSerializer( - sizePer, - new GenericIndexedWriter>( - ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) - ), - compression - ); - return retVal; - } - + private final IOPeon ioPeon; private final int sizePer; private final GenericIndexedWriter> flattener; private final CompressedObjectStrategy.CompressionStrategy compression; + private final String metaFile; + private long metaCount = 0; private int numInserted = 0; - private FloatBuffer endBuffer; - public CompressedFloatsSupplierSerializer( - int sizePer, - GenericIndexedWriter> flattener, + public BlockLayoutFloatSupplierSerializer( + IOPeon ioPeon, + String filenameBase, + ByteOrder order, CompressedObjectStrategy.CompressionStrategy compression ) { - this.sizePer = sizePer; - this.flattener = flattener; + this.ioPeon = ioPeon; + this.sizePer = CompressedPools.BUFFER_SIZE / Floats.BYTES; + this.flattener = new GenericIndexedWriter<>( + ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + ); + this.metaFile = filenameBase + ".format"; this.compression = compression; endBuffer = FloatBuffer.allocate(sizePer); endBuffer.mark(); } + @Override public void open() throws IOException { flattener.open(); } + @Override public int size() { return numInserted; } + @Override public void add(float value) throws IOException { if (!endBuffer.hasRemaining()) { @@ -111,18 +95,18 @@ public void add(float value) throws IOException ++numInserted; } - public void closeAndConsolidate(OutputSupplier consolidatedOut) throws IOException + @Override + public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException { close(); - try (OutputStream out = consolidatedOut.getOutput()) { - out.write(CompressedFloatsIndexedSupplier.version); - out.write(Ints.toByteArray(numInserted)); - out.write(Ints.toByteArray(sizePer)); - out.write(new byte[]{compression.getId()}); - ByteStreams.copy(flattener.combineStreams(), out); + try (OutputStream out = consolidatedOut.openStream(); + InputStream meta = ioPeon.makeInputStream(metaFile)) { + ByteStreams.copy(meta, out); + flattener.combineStreams().copyTo(out); } } + @Override public void close() throws IOException { endBuffer.limit(endBuffer.position()); @@ -130,24 +114,31 @@ public void close() throws IOException flattener.write(StupidResourceHolder.create(endBuffer)); endBuffer = null; flattener.close(); + + try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { + metaOut.write(CompressedFloatsIndexedSupplier.version); + metaOut.write(Ints.toByteArray(numInserted)); + metaOut.write(Ints.toByteArray(sizePer)); + metaOut.write(compression.getId()); + metaOut.close(); + metaCount = metaOut.getCount(); + } } + @Override public long getSerializedSize() { - return 1 + // version - Ints.BYTES + // elements num - Ints.BYTES + // sizePer - 1 + // compression id - flattener.getSerializedSize(); + return metaCount + flattener.getSerializedSize(); } + @Override public void writeToChannel(WritableByteChannel channel) throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{CompressedFloatsIndexedSupplier.version})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); - ByteStreams.copy(from, channel); + try (InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream input = flattener.combineStreams().openStream()) { + ByteStreams.copy(Channels.newChannel(meta), channel); + final ReadableByteChannel from = Channels.newChannel(input); + ByteStreams.copy(from, channel); + } } } diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java new file mode 100644 index 000000000000..2e54eecd8ff6 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java @@ -0,0 +1,148 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Supplier; +import com.google.common.io.Closeables; +import com.google.common.primitives.Floats; +import com.metamx.common.guava.CloseQuietly; +import io.druid.collections.ResourceHolder; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.FloatBuffer; + +public class BlockLayoutIndexedFloatSupplier implements Supplier +{ + private final GenericIndexed> baseFloatBuffers; + private final int totalSize; + private final int sizePer; + + public BlockLayoutIndexedFloatSupplier( + int totalSize, int sizePer, ByteBuffer fromBuffer, ByteOrder order, + CompressedObjectStrategy.CompressionStrategy strategy + ) + { + baseFloatBuffers = GenericIndexed.read(fromBuffer, VSizeCompressedObjectStrategy.getBufferForOrder( + order, strategy, sizePer * Floats.BYTES + )); + this.totalSize = totalSize; + this.sizePer = sizePer; + } + + @Override + public IndexedFloats get() + { + final int div = Integer.numberOfTrailingZeros(sizePer); + final int rem = sizePer - 1; + final boolean powerOf2 = sizePer == (1 << div); + if (powerOf2) { + return new BlockLayoutIndexedFloats() + { + @Override + public float get(int index) + { + // optimize division and remainder for powers of 2 + final int bufferNum = index >> div; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + final int bufferIndex = index & rem; + return floatBuffer.get(floatBuffer.position() + bufferIndex); + } + }; + } else { + return new BlockLayoutIndexedFloats(); + } + } + + private class BlockLayoutIndexedFloats implements IndexedFloats + { + final Indexed> singleThreadedFloatBuffers = baseFloatBuffers.singleThreaded(); + int currIndex = -1; + ResourceHolder holder; + ByteBuffer buffer; + FloatBuffer floatBuffer; + + @Override + public int size() + { + return totalSize; + } + + @Override + public float get(int index) + { + // division + remainder is optimized by the compiler so keep those together + final int bufferNum = index / sizePer; + final int bufferIndex = index % sizePer; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + return floatBuffer.get(floatBuffer.position() + bufferIndex); + } + + @Override + public void fill(int index, float[] toFill) + { + if (totalSize - index < toFill.length) { + throw new IndexOutOfBoundsException( + String.format( + "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize + ) + ); + } + for (int i = 0; i < toFill.length; i++) { + toFill[i] = get(index + i); + } + } + + protected void loadBuffer(int bufferNum) + { + CloseQuietly.close(holder); + holder = singleThreadedFloatBuffers.get(bufferNum); + buffer = holder.get(); + floatBuffer = buffer.asFloatBuffer(); + currIndex = bufferNum; + } + + @Override + public String toString() + { + return "BlockCompressedIndexedFloats_Anonymous{" + + "currIndex=" + currIndex + + ", sizePer=" + sizePer + + ", numChunks=" + singleThreadedFloatBuffers.size() + + ", totalSize=" + totalSize + + '}'; + } + + @Override + public void close() throws IOException + { + Closeables.close(holder, false); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java new file mode 100644 index 000000000000..fd7ceaae9e4c --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java @@ -0,0 +1,183 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Supplier; +import com.google.common.io.Closeables; +import com.metamx.common.guava.CloseQuietly; +import io.druid.collections.ResourceHolder; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.LongBuffer; + +public class BlockLayoutIndexedLongSupplier implements Supplier +{ + + private final GenericIndexed> baseLongBuffers; + private final int totalSize; + private final int sizePer; + private final CompressionFactory.LongEncodingReader baseReader; + + public BlockLayoutIndexedLongSupplier( + int totalSize, int sizePer, ByteBuffer fromBuffer, ByteOrder order, + CompressionFactory.LongEncodingReader reader, + CompressedObjectStrategy.CompressionStrategy strategy + ) + { + baseLongBuffers = GenericIndexed.read(fromBuffer, VSizeCompressedObjectStrategy.getBufferForOrder( + order, strategy, reader.getNumBytes(sizePer) + )); + this.totalSize = totalSize; + this.sizePer = sizePer; + this.baseReader = reader; + } + + @Override + public IndexedLongs get() + { + final int div = Integer.numberOfTrailingZeros(sizePer); + final int rem = sizePer - 1; + final boolean powerOf2 = sizePer == (1 << div); + if (powerOf2) { + // this provide slightly better performance than calling the LongsEncodingReader.read, probably because Java + // doesn't inline the method call for some reason. This should be removed when test show that performance + // of using read method is same as directly accessing the longbuffer + if (baseReader instanceof LongsLongEncodingReader) { + return new BlockLayoutIndexedLongs() + { + @Override + public long get(int index) + { + // optimize division and remainder for powers of 2 + final int bufferNum = index >> div; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + final int bufferIndex = index & rem; + return longBuffer.get(longBuffer.position() + bufferIndex); + } + + protected void loadBuffer(int bufferNum) + { + CloseQuietly.close(holder); + holder = singleThreadedLongBuffers.get(bufferNum); + buffer = holder.get(); + longBuffer = buffer.asLongBuffer(); + currIndex = bufferNum; + } + }; + } else { + return new BlockLayoutIndexedLongs() + { + @Override + public long get(int index) + { + // optimize division and remainder for powers of 2 + final int bufferNum = index >> div; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + final int bufferIndex = index & rem; + return reader.read(bufferIndex); + } + }; + } + + } else { + return new BlockLayoutIndexedLongs(); + } + } + + private class BlockLayoutIndexedLongs implements IndexedLongs + { + final CompressionFactory.LongEncodingReader reader = baseReader.duplicate(); + final Indexed> singleThreadedLongBuffers = baseLongBuffers.singleThreaded(); + int currIndex = -1; + ResourceHolder holder; + ByteBuffer buffer; + LongBuffer longBuffer; + + @Override + public int size() + { + return totalSize; + } + + @Override + public long get(int index) + { + final int bufferNum = index / sizePer; + final int bufferIndex = index % sizePer; + + if (bufferNum != currIndex) { + loadBuffer(bufferNum); + } + + return reader.read(bufferIndex); + } + + @Override + public void fill(int index, long[] toFill) + { + if (totalSize - index < toFill.length) { + throw new IndexOutOfBoundsException( + String.format( + "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize + ) + ); + } + for (int i = 0; i < toFill.length; i++) { + toFill[i] = get(index + i); + } + } + + protected void loadBuffer(int bufferNum) + { + CloseQuietly.close(holder); + holder = singleThreadedLongBuffers.get(bufferNum); + buffer = holder.get(); + currIndex = bufferNum; + reader.setBuffer(buffer); + } + + @Override + public String toString() + { + return "BlockCompressedIndexedLongs_Anonymous{" + + "currIndex=" + currIndex + + ", sizePer=" + sizePer + + ", numChunks=" + singleThreadedLongBuffers.size() + + ", totalSize=" + totalSize + + '}'; + } + + @Override + public void close() throws IOException + { + Closeables.close(holder, false); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java new file mode 100644 index 000000000000..6554ba508782 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java @@ -0,0 +1,157 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.io.ByteSink; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; +import com.google.common.primitives.Ints; +import io.druid.collections.ResourceHolder; +import io.druid.collections.StupidResourceHolder; +import io.druid.segment.CompressedPools; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; + +public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer +{ + + private final IOPeon ioPeon; + private final int sizePer; + private final CompressionFactory.LongEncodingWriter writer; + private final GenericIndexedWriter> flattener; + private final CompressedObjectStrategy.CompressionStrategy compression; + private final String metaFile; + private long metaCount = 0; + + private int numInserted = 0; + + private ByteBuffer endBuffer = null; + + public BlockLayoutLongSupplierSerializer( + IOPeon ioPeon, + String filenameBase, + ByteOrder order, + CompressionFactory.LongEncodingWriter writer, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.ioPeon = ioPeon; + this.sizePer = writer.getBlockSize(CompressedPools.BUFFER_SIZE); + this.flattener = new GenericIndexedWriter>( + ioPeon, + filenameBase, + VSizeCompressedObjectStrategy.getBufferForOrder( + order, + compression, + writer.getNumBytes(sizePer) + ) + ); + this.metaFile = filenameBase + ".format"; + this.writer = writer; + this.compression = compression; + } + + @Override + public void open() throws IOException + { + flattener.open(); + } + + @Override + public int size() + { + return numInserted; + } + + @Override + public void add(long value) throws IOException + { + if (numInserted % sizePer == 0) { + if (endBuffer != null) { + writer.flush(); + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + } + endBuffer = ByteBuffer.allocate(writer.getNumBytes(sizePer)); + writer.setBuffer(endBuffer); + } + + writer.write(value); + ++numInserted; + } + + @Override + public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + { + close(); + try (OutputStream out = consolidatedOut.openStream(); + InputStream meta = ioPeon.makeInputStream(metaFile)) { + ByteStreams.copy(meta, out); + flattener.combineStreams().copyTo(out); + } + } + + @Override + public void close() throws IOException + { + if (endBuffer != null) { + writer.flush(); + endBuffer.limit(endBuffer.position()); + endBuffer.rewind(); + flattener.write(StupidResourceHolder.create(endBuffer)); + } + endBuffer = null; + flattener.close(); + + try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { + metaOut.write(CompressedLongsIndexedSupplier.version); + metaOut.write(Ints.toByteArray(numInserted)); + metaOut.write(Ints.toByteArray(sizePer)); + writer.putMeta(metaOut, compression); + metaOut.close(); + metaCount = metaOut.getCount(); + } + } + + @Override + public long getSerializedSize() + { + return metaCount + flattener.getSerializedSize(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + try (InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream input = flattener.combineStreams().openStream()) { + ByteStreams.copy(Channels.newChannel(meta), channel); + final ReadableByteChannel from = Channels.newChannel(input); + ByteStreams.copy(from, channel); + } + } +} \ No newline at end of file diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java index b60dd0a9620d..cf46bcb9cb48 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -19,47 +19,38 @@ package io.druid.segment.data; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import com.google.common.io.Closeables; -import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.metamx.common.IAE; -import com.metamx.common.guava.CloseQuietly; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; -import io.druid.segment.CompressedPools; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.FloatBuffer; import java.nio.channels.WritableByteChannel; -import java.util.Iterator; -/** - */ public class CompressedFloatsIndexedSupplier implements Supplier { public static final byte LZF_VERSION = 0x1; public static final byte version = 0x2; - public static final int MAX_FLOATS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Floats.BYTES; private final int totalSize; private final int sizePer; - private final GenericIndexed> baseFloatBuffers; + private final ByteBuffer buffer; + private final Supplier supplier; private final CompressedObjectStrategy.CompressionStrategy compression; CompressedFloatsIndexedSupplier( int totalSize, int sizePer, - GenericIndexed> baseFloatBuffers, + ByteBuffer buffer, + Supplier supplier, CompressedObjectStrategy.CompressionStrategy compression ) { this.totalSize = totalSize; this.sizePer = sizePer; - this.baseFloatBuffers = baseFloatBuffers; + this.buffer = buffer; + this.supplier = supplier; this.compression = compression; } @@ -71,33 +62,12 @@ public int size() @Override public IndexedFloats get() { - final int div = Integer.numberOfTrailingZeros(sizePer); - final int rem = sizePer - 1; - final boolean powerOf2 = sizePer == (1 << div); - if(powerOf2) { - return new CompressedIndexedFloats() { - @Override - public float get(int index) - { - // optimize division and remainder for powers of 2 - final int bufferNum = index >> div; - - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - final int bufferIndex = index & rem; - return buffer.get(buffer.position() + bufferIndex); - } - }; - } else { - return new CompressedIndexedFloats(); - } + return supplier.get(); } public long getSerializedSize() { - return baseFloatBuffers.getSerializedSize() + 1 + 4 + 4 + 1; + return buffer.remaining() + 1 + 4 + 4 + 1; } public void writeToChannel(WritableByteChannel channel) throws IOException @@ -106,212 +76,37 @@ public void writeToChannel(WritableByteChannel channel) throws IOException channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - baseFloatBuffers.writeToChannel(channel); - } - - public CompressedFloatsIndexedSupplier convertByteOrder(ByteOrder order) - { - return new CompressedFloatsIndexedSupplier( - totalSize, - sizePer, - GenericIndexed.fromIterable(baseFloatBuffers, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), - compression - ); - } - - /** - * For testing. Do not depend on unless you like things breaking. - */ - GenericIndexed> getBaseFloatBuffers() - { - return baseFloatBuffers; + channel.write(buffer.asReadOnlyBuffer()); } public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); - if (versionFromBuffer == version) { + if (versionFromBuffer == LZF_VERSION || versionFromBuffer == version) { final int totalSize = buffer.getInt(); final int sizePer = buffer.getInt(); - final CompressedObjectStrategy.CompressionStrategy compression = - CompressedObjectStrategy.CompressionStrategy.forId(buffer.get()); - - return new CompressedFloatsIndexedSupplier( + CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; + if (versionFromBuffer == version) { + byte compressionId = buffer.get(); + compression = CompressedObjectStrategy.CompressionStrategy.forId(compressionId); + } + Supplier supplier = CompressionFactory.getFloatSupplier( totalSize, sizePer, - GenericIndexed.read( - buffer, - CompressedFloatBufferObjectStrategy.getBufferForOrder( - order, - compression, - sizePer - ) - ), + buffer.asReadOnlyBuffer(), + order, compression ); - } else if (versionFromBuffer == LZF_VERSION) { - final int totalSize = buffer.getInt(); - final int sizePer = buffer.getInt(); - final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; return new CompressedFloatsIndexedSupplier( totalSize, sizePer, - GenericIndexed.read( - buffer, - CompressedFloatBufferObjectStrategy.getBufferForOrder( - order, - compression, - sizePer - ) - ), + buffer, + supplier, compression ); } throw new IAE("Unknown version[%s]", versionFromBuffer); } - - public static CompressedFloatsIndexedSupplier fromFloatBuffer(FloatBuffer buffer, final ByteOrder order, CompressedObjectStrategy.CompressionStrategy compression) - { - return fromFloatBuffer(buffer, MAX_FLOATS_IN_BUFFER, order, compression); - } - - public static CompressedFloatsIndexedSupplier fromFloatBuffer( - final FloatBuffer buffer, final int chunkFactor, final ByteOrder order, final CompressedObjectStrategy.CompressionStrategy compression - ) - { - Preconditions.checkArgument( - chunkFactor <= MAX_FLOATS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor - ); - - return new CompressedFloatsIndexedSupplier( - buffer.remaining(), - chunkFactor, - GenericIndexed.fromIterable( - new Iterable>() - { - @Override - public Iterator> iterator() - { - return new Iterator>() - { - FloatBuffer myBuffer = buffer.asReadOnlyBuffer(); - - @Override - public boolean hasNext() - { - return myBuffer.hasRemaining(); - } - - @Override - public ResourceHolder next() - { - final FloatBuffer retVal = myBuffer.asReadOnlyBuffer(); - - if (chunkFactor < myBuffer.remaining()) { - retVal.limit(retVal.position() + chunkFactor); - } - myBuffer.position(myBuffer.position() + retVal.remaining()); - - return StupidResourceHolder.create(retVal); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } - }, - CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, chunkFactor) - ), - compression - ); - } - - private class CompressedIndexedFloats implements IndexedFloats - { - final Indexed> singleThreadedFloatBuffers = baseFloatBuffers.singleThreaded(); - - int currIndex = -1; - ResourceHolder holder; - FloatBuffer buffer; - - @Override - public int size() - { - return totalSize; - } - - @Override - public float get(final int index) - { - // division + remainder is optimized by the compiler so keep those together - final int bufferNum = index / sizePer; - final int bufferIndex = index % sizePer; - - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - return buffer.get(buffer.position() + bufferIndex); - } - - @Override - public void fill(int index, float[] toFill) - { - if (totalSize - index < toFill.length) { - throw new IndexOutOfBoundsException( - String.format( - "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize - ) - ); - } - - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - int leftToFill = toFill.length; - while (leftToFill > 0) { - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - buffer.mark(); - buffer.position(buffer.position() + bufferIndex); - final int numToGet = Math.min(buffer.remaining(), leftToFill); - buffer.get(toFill, toFill.length - leftToFill, numToGet); - buffer.reset(); - leftToFill -= numToGet; - ++bufferNum; - bufferIndex = 0; - } - } - - protected void loadBuffer(int bufferNum) - { - CloseQuietly.close(holder); - holder = singleThreadedFloatBuffers.get(bufferNum); - buffer = holder.get(); - currIndex = bufferNum; - } - - @Override - public String toString() - { - return "CompressedFloatsIndexedSupplier_Anonymous{" + - "currIndex=" + currIndex + - ", sizePer=" + sizePer + - ", numChunks=" + singleThreadedFloatBuffers.size() + - ", totalSize=" + totalSize + - '}'; - } - - @Override - public void close() throws IOException - { - Closeables.close(holder, false); - } - } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java index 2ac0b4ed465f..d4117f94ae02 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -19,24 +19,14 @@ package io.druid.segment.data; -import com.google.common.base.Preconditions; import com.google.common.base.Supplier; -import com.google.common.io.Closeables; import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; import com.metamx.common.IAE; -import com.metamx.common.guava.CloseQuietly; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; -import io.druid.segment.CompressedPools; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.LongBuffer; import java.nio.channels.WritableByteChannel; -import java.util.Iterator; -import java.util.List; /** */ @@ -44,25 +34,30 @@ public class CompressedLongsIndexedSupplier implements Supplier { public static final byte LZF_VERSION = 0x1; public static final byte version = 0x2; - public static final int MAX_LONGS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Longs.BYTES; private final int totalSize; private final int sizePer; - private final GenericIndexed> baseLongBuffers; + private final ByteBuffer buffer; + private final Supplier supplier; private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionFactory.LongEncodingFormat encoding; CompressedLongsIndexedSupplier( int totalSize, int sizePer, - GenericIndexed> baseLongBuffers, - CompressedObjectStrategy.CompressionStrategy compression + ByteBuffer buffer, + Supplier supplier, + CompressedObjectStrategy.CompressionStrategy compression, + CompressionFactory.LongEncodingFormat encoding ) { this.totalSize = totalSize; this.sizePer = sizePer; - this.baseLongBuffers = baseLongBuffers; + this.buffer = buffer; + this.supplier = supplier; this.compression = compression; + this.encoding = encoding; } public int size() @@ -73,33 +68,12 @@ public int size() @Override public IndexedLongs get() { - final int div = Integer.numberOfTrailingZeros(sizePer); - final int rem = sizePer - 1; - final boolean powerOf2 = sizePer == (1 << div); - if(powerOf2) { - return new CompressedIndexedLongs() { - @Override - public long get(int index) - { - // optimize division and remainder for powers of 2 - final int bufferNum = index >> div; - - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - final int bufferIndex = index & rem; - return buffer.get(buffer.position() + bufferIndex); - } - }; - } else { - return new CompressedIndexedLongs(); - } + return supplier.get(); } public long getSerializedSize() { - return baseLongBuffers.getSerializedSize() + 1 + 4 + 4 + 1; + return buffer.remaining() + 1 + 4 + 4 + 1 + (encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT ? 0 : 1); } public void writeToChannel(WritableByteChannel channel) throws IOException @@ -107,268 +81,50 @@ public void writeToChannel(WritableByteChannel channel) throws IOException channel.write(ByteBuffer.wrap(new byte[]{version})); channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - baseLongBuffers.writeToChannel(channel); - } - - public CompressedLongsIndexedSupplier convertByteOrder(ByteOrder order) - { - return new CompressedLongsIndexedSupplier( - totalSize, - sizePer, - GenericIndexed.fromIterable(baseLongBuffers, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), - compression - ); - } - - /** - * For testing. Do not use unless you like things breaking - */ - GenericIndexed> getBaseLongBuffers() - { - return baseLongBuffers; + if (encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT) { + channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + } else { + channel.write(ByteBuffer.wrap(new byte[]{CompressionFactory.setEncodingFlag(compression.getId())})); + channel.write(ByteBuffer.wrap(new byte[]{encoding.getId()})); + } + channel.write(buffer.asReadOnlyBuffer()); } public static CompressedLongsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); - if (versionFromBuffer == version) { + if (versionFromBuffer == LZF_VERSION || versionFromBuffer == version) { final int totalSize = buffer.getInt(); final int sizePer = buffer.getInt(); - final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId(buffer.get()); - return new CompressedLongsIndexedSupplier( + CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; + CompressionFactory.LongEncodingFormat encoding = CompressionFactory.LEGACY_LONG_ENCODING_FORMAT; + if (versionFromBuffer == version) { + byte compressionId = buffer.get(); + if (CompressionFactory.hasEncodingFlag(compressionId)) { + encoding = CompressionFactory.LongEncodingFormat.forId(buffer.get()); + compressionId = CompressionFactory.clearEncodingFlag(compressionId); + } + compression = CompressedObjectStrategy.CompressionStrategy.forId(compressionId); + } + Supplier supplier = CompressionFactory.getLongSupplier( totalSize, sizePer, - GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), - compression + buffer.asReadOnlyBuffer(), + order, + encoding, + compression ); - } else if (versionFromBuffer == LZF_VERSION) { - final int totalSize = buffer.getInt(); - final int sizePer = buffer.getInt(); - final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; return new CompressedLongsIndexedSupplier( totalSize, sizePer, - GenericIndexed.read(buffer, CompressedLongBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), - compression + buffer, + supplier, + compression, + encoding ); } throw new IAE("Unknown version[%s]", versionFromBuffer); } - - public static CompressedLongsIndexedSupplier fromLongBuffer(LongBuffer buffer, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression) - { - return fromLongBuffer(buffer, MAX_LONGS_IN_BUFFER, byteOrder, compression); - } - - public static CompressedLongsIndexedSupplier fromLongBuffer( - final LongBuffer buffer, final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression - ) - { - Preconditions.checkArgument( - chunkFactor <= MAX_LONGS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor - ); - - return new CompressedLongsIndexedSupplier( - buffer.remaining(), - chunkFactor, - GenericIndexed.fromIterable( - new Iterable>() - { - @Override - public Iterator> iterator() - { - return new Iterator>() - { - LongBuffer myBuffer = buffer.asReadOnlyBuffer(); - - @Override - public boolean hasNext() - { - return myBuffer.hasRemaining(); - } - - @Override - public ResourceHolder next() - { - LongBuffer retVal = myBuffer.asReadOnlyBuffer(); - - if (chunkFactor < myBuffer.remaining()) { - retVal.limit(retVal.position() + chunkFactor); - } - myBuffer.position(myBuffer.position() + retVal.remaining()); - - return StupidResourceHolder.create(retVal); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } - }, - CompressedLongBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) - ), - compression - ); - } - - public static CompressedLongsIndexedSupplier fromList( - final List list , final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression - ) - { - Preconditions.checkArgument( - chunkFactor <= MAX_LONGS_IN_BUFFER, "Chunks must be <= 64k bytes. chunkFactor was[%s]", chunkFactor - ); - - return new CompressedLongsIndexedSupplier( - list.size(), - chunkFactor, - GenericIndexed.fromIterable( - new Iterable>() - { - @Override - public Iterator> iterator() - { - return new Iterator>() - { - int position = 0; - - @Override - public boolean hasNext() - { - return position < list.size(); - } - - @Override - public ResourceHolder next() - { - LongBuffer retVal = LongBuffer.allocate(chunkFactor); - - if (chunkFactor > list.size() - position) { - retVal.limit(list.size() - position); - } - final List longs = list.subList(position, position + retVal.remaining()); - for (long value : longs) { - retVal.put(value); - } - retVal.rewind(); - position += retVal.remaining(); - - return StupidResourceHolder.create(retVal); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - }; - } - }, - CompressedLongBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) - ), - compression - ); - } - - private class CompressedIndexedLongs implements IndexedLongs - { - final Indexed> singleThreadedLongBuffers = baseLongBuffers.singleThreaded(); - - int currIndex = -1; - ResourceHolder holder; - LongBuffer buffer; - - @Override - public int size() - { - return totalSize; - } - - @Override - public long get(int index) - { - final int bufferNum = index / sizePer; - final int bufferIndex = index % sizePer; - - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - return buffer.get(buffer.position() + bufferIndex); - } - - @Override - public void fill(int index, long[] toFill) - { - if (totalSize - index < toFill.length) { - throw new IndexOutOfBoundsException( - String.format( - "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize - ) - ); - } - - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - int leftToFill = toFill.length; - while (leftToFill > 0) { - if (bufferNum != currIndex) { - loadBuffer(bufferNum); - } - - buffer.mark(); - buffer.position(buffer.position() + bufferIndex); - final int numToGet = Math.min(buffer.remaining(), leftToFill); - buffer.get(toFill, toFill.length - leftToFill, numToGet); - buffer.reset(); - leftToFill -= numToGet; - ++bufferNum; - bufferIndex = 0; - } - } - - protected void loadBuffer(int bufferNum) - { - CloseQuietly.close(holder); - holder = singleThreadedLongBuffers.get(bufferNum); - buffer = holder.get(); - currIndex = bufferNum; - } - - @Override - public int binarySearch(long key) - { - throw new UnsupportedOperationException(); - } - - @Override - public int binarySearch(long key, int from, int to) - { - throw new UnsupportedOperationException(); - } - - @Override - public String toString() - { - return "CompressedLongsIndexedSupplier_Anonymous{" + - "currIndex=" + currIndex + - ", sizePer=" + sizePer + - ", numChunks=" + singleThreadedLongBuffers.size() + - ", totalSize=" + totalSize + - '}'; - } - - @Override - public void close() throws IOException - { - Closeables.close(holder, false); - } - } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java deleted file mode 100644 index 5ea2b20bcc3c..000000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsSupplierSerializer.java +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import com.google.common.io.ByteStreams; -import com.google.common.io.OutputSupplier; -import com.google.common.primitives.Ints; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; - -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.LongBuffer; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; - -/** - */ -public class CompressedLongsSupplierSerializer -{ - public static CompressedLongsSupplierSerializer create( - final IOPeon ioPeon, - final String filenameBase, - final ByteOrder order, - final CompressedObjectStrategy.CompressionStrategy compression - ) throws IOException - { - final CompressedLongsSupplierSerializer retVal = new CompressedLongsSupplierSerializer( - CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER, - new GenericIndexedWriter>( - ioPeon, - filenameBase, - CompressedLongBufferObjectStrategy.getBufferForOrder( - order, - compression, - CompressedLongsIndexedSupplier.MAX_LONGS_IN_BUFFER - ) - ), - compression - ); - return retVal; - } - - private final int sizePer; - private final GenericIndexedWriter> flattener; - private final CompressedObjectStrategy.CompressionStrategy compression; - - private int numInserted = 0; - - private LongBuffer endBuffer; - - public CompressedLongsSupplierSerializer( - int sizePer, - GenericIndexedWriter> flattener, - CompressedObjectStrategy.CompressionStrategy compression - ) - { - this.sizePer = sizePer; - this.flattener = flattener; - this.compression = compression; - - endBuffer = LongBuffer.allocate(sizePer); - endBuffer.mark(); - } - - public void open() throws IOException - { - flattener.open(); - } - - public int size() - { - return numInserted; - } - - public void add(long value) throws IOException - { - if (!endBuffer.hasRemaining()) { - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = LongBuffer.allocate(sizePer); - endBuffer.mark(); - } - - endBuffer.put(value); - ++numInserted; - } - - public void closeAndConsolidate(OutputSupplier consolidatedOut) throws IOException - { - close(); - try (OutputStream out = consolidatedOut.getOutput()) { - out.write(CompressedLongsIndexedSupplier.version); - out.write(Ints.toByteArray(numInserted)); - out.write(Ints.toByteArray(sizePer)); - out.write(new byte[]{compression.getId()}); - ByteStreams.copy(flattener.combineStreams(), out); - } - } - - public void close() throws IOException { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = null; - flattener.close(); - } - - public long getSerializedSize() - { - return 1 + // version - Ints.BYTES + // elements num - Ints.BYTES + // sizePer - 1 + // compression id - flattener.getSerializedSize(); - } - - public void writeToChannel(WritableByteChannel channel) throws IOException - { - channel.write(ByteBuffer.wrap(new byte[]{CompressedFloatsIndexedSupplier.version})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - final ReadableByteChannel from = Channels.newChannel(flattener.combineStreams().getInput()); - ByteStreams.copy(from, channel); - } -} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java index c6dd4f9b3781..fac137c4a291 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java @@ -19,6 +19,8 @@ package io.druid.segment.data; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Throwables; import com.google.common.collect.Maps; import com.metamx.common.logger.Logger; @@ -30,6 +32,7 @@ import net.jpountz.lz4.LZ4Factory; import net.jpountz.lz4.LZ4FastDecompressor; import net.jpountz.lz4.LZ4SafeDecompressor; +import org.apache.commons.lang.ArrayUtils; import java.io.IOException; import java.nio.Buffer; @@ -44,6 +47,13 @@ public class CompressedObjectStrategy implements ObjectStrateg private static final Logger log = new Logger(CompressedObjectStrategy.class); public static final CompressionStrategy DEFAULT_COMPRESSION_STRATEGY = CompressionStrategy.LZ4; + /** + * Compression strategy is used to compress block of bytes without knowledge of what data the bytes represents. + * + * When adding compression strategy, do not use id in the range [0x7C, 0xFD] (greater than 123 or less than -2), since + * a flag mechanism is used in CompressionFactory that involves subtracting the value 126 from the compression id + * (see {@link CompressionFactory#FLAG_BOUND}) + */ public static enum CompressionStrategy { LZF((byte) 0x0) { @@ -85,6 +95,23 @@ public Compressor getCompressor() { return UncompressedCompressor.defaultCompressor; } + }, + /* + This value indicate no compression strategy should be used, and compression should not be block based + Currently only IndexedLong support non block based compression, and other types treat this as UNCOMPRESSED + */ + NONE((byte) 0xFE) { + @Override + public Decompressor getDecompressor() + { + throw new UnsupportedOperationException("NONE compression strategy shouldn't use any decompressor"); + } + + @Override + public Compressor getCompressor() + { + throw new UnsupportedOperationException("NONE compression strategy shouldn't use any compressor"); + } }; final byte id; @@ -103,6 +130,19 @@ public byte getId() public abstract Decompressor getDecompressor(); + @JsonValue + @Override + public String toString() + { + return this.name().toLowerCase(); + } + + @JsonCreator + public static CompressionStrategy fromString(String name) + { + return valueOf(name.toUpperCase()); + } + static final Map idMap = Maps.newHashMap(); static { @@ -115,6 +155,12 @@ public static CompressionStrategy forId(byte id) { return idMap.get(id); } + + // TODO remove this method and change all its callers to use all CompressionStrategy values when NONE type is supported by all types + public static CompressionStrategy[] noNoneValues() + { + return (CompressionStrategy[]) ArrayUtils.removeElement(CompressionStrategy.values(), NONE); + } } public static interface Decompressor diff --git a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java new file mode 100644 index 000000000000..1f0958f7cf52 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java @@ -0,0 +1,333 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.base.Supplier; +import com.google.common.collect.Maps; +import com.metamx.common.IAE; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; + +/** + * Compression of metrics is done by using a combination of {@link io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy} + * and Encoding(such as {@link LongEncodingStrategy} for type Long). CompressionStrategy is unaware of the data type + * and is based on byte operations. It must compress and decompress in block of bytes. Encoding refers to compression + * method relies on data format, so a different set of Encodings exist for each data type. + *

+ * Storage Format : + * Byte 1 : version (currently 0x02) + * Byte 2 - 5 : number of values + * Byte 6 - 9 : size per block (even if block format isn't used, this is needed for backward compatibility) + * Byte 10 : compression strategy (contains a flag if there's an encoding byte, see below for how the flag is defined) + * Byte 11(optional) : encoding type + *

+ * Encoding specific header (described below) + *

+ * Block related header (if block compression is used, described in GenericIndexed) + *

+ * Values + */ +public class CompressionFactory +{ + private CompressionFactory() + { + // No instantiation + } + + public static final LongEncodingStrategy DEFAULT_LONG_ENCODING_STRATEGY = LongEncodingStrategy.LONGS; + + // encoding format for segments created prior to the introduction of encoding formats + public static final LongEncodingFormat LEGACY_LONG_ENCODING_FORMAT = LongEncodingFormat.LONGS; + + /* + * Delta Encoding Header v1: + * Byte 1 : version + * Byte 2 - 9 : base value + * Byte 10 - 13 : number of bits per value + */ + public static final byte DELTA_ENCODING_VERSION = 0x1; + + /* + * Table Encoding Header v1 : + * Byte 1 : version + * Byte 2 - 5 : table size + * Byte 6 - (6 + 8 * table size - 1) : table of encoding, where the ith 8-byte value is encoded as i + */ + public static final byte TABLE_ENCODING_VERSION = 0x1; + + public static final int MAX_TABLE_SIZE = 256; + + /* + * There is no header or version for Longs encoding for backward compatibility + */ + + /* + * This is the flag mechanism for determine whether an encoding byte exist in the header. This is needed for + * backward compatibility, since segment created prior to the introduction of encoding formats does not have the + * encoding strategy byte. The flag is encoded in the compression strategy byte using the setEncodingFlag and + * clearEncodingFlag function. + */ + + // 0xFE(-2) should be the smallest valid compression strategy id + private static byte FLAG_BOUND = (byte) 0xFE; + // 126 is the value here since -2 - 126 = -128, which is the lowest byte value + private static int FLAG_VALUE = 126; + + public static boolean hasEncodingFlag(byte strategyId) + { + return strategyId < FLAG_BOUND; + } + + public static byte setEncodingFlag(byte strategyId) + { + return hasEncodingFlag(strategyId) ? strategyId : (byte) (strategyId - FLAG_VALUE); + } + + public static byte clearEncodingFlag(byte strategyId) + { + return hasEncodingFlag(strategyId) ? (byte) (strategyId + FLAG_VALUE) : strategyId; + } + + /* + * The compression of decompression of encodings are separated into different enums. EncodingStrategy refers to the + * strategy used to encode the data, and EncodingFormat refers to the format the data is encoded in. Note there is not + * necessarily an one-to-one mapping between to two. For instance, the AUTO LongEncodingStrategy scans the data once + * and decide on which LongEncodingFormat to use based on data property, so it's possible for the EncodingStrategy to + * write in any of the LongEncodingFormat. On the other hand, there are no LongEncodingStrategy that always write in + * TABLE LongEncodingFormat since it only works for data with low cardinality. + */ + + public enum LongEncodingStrategy + { + /** + * AUTO strategy scans all values once before encoding them. It stores the value cardinality and maximum offset + * of the values to determine whether to use DELTA, TABLE, or LONGS format. + */ + AUTO, + + /** + * LONGS strategy always encode the values using LONGS format + */ + LONGS; + + @JsonValue + @Override + public String toString() + { + return this.name().toLowerCase(); + } + + @JsonCreator + public static LongEncodingStrategy fromString(String name) + { + return valueOf(name.toUpperCase()); + } + } + + public enum LongEncodingFormat + { + /** + * DELTA format encodes a series of longs by finding the smallest value first, and stores all values + * as offset to the smallest value. The maximum value is also found to calculate how many bits are required + * to store each offset using {@link VSizeLongSerde}. + */ + DELTA((byte) 0x0) { + @Override + public LongEncodingReader getReader(ByteBuffer buffer, ByteOrder order) + { + return new DeltaLongEncodingReader(buffer); + } + }, + /** + * TABLE format encodes a series of longs by mapping each unique value to an id, and string the id with the + * minimum number of bits similar to how DELTA stores offset. TABLE format is only applicable to values with + * less unique values than {@link CompressionFactory#MAX_TABLE_SIZE}. + */ + TABLE((byte) 0x1) { + @Override + public LongEncodingReader getReader(ByteBuffer buffer, ByteOrder order) + { + return new TableLongEncodingReader(buffer); + } + }, + /** + * LONGS format encodes longs as is, using 8 bytes for each value. + */ + LONGS((byte) 0xFF) { + @Override + public LongEncodingReader getReader(ByteBuffer buffer, ByteOrder order) + { + return new LongsLongEncodingReader(buffer, order); + } + }; + + final byte id; + + LongEncodingFormat(byte id) + { + this.id = id; + } + + public byte getId() + { + return id; + } + + static final Map idMap = Maps.newHashMap(); + + static { + for (LongEncodingFormat format : LongEncodingFormat.values()) { + idMap.put(format.getId(), format); + } + } + + public abstract LongEncodingReader getReader(ByteBuffer buffer, ByteOrder order); + + public static LongEncodingFormat forId(byte id) + { + return idMap.get(id); + } + } + + /** + * This writer output encoded values to the given ByteBuffer or OutputStream. {@link #setBuffer(ByteBuffer)} or + * {@link #setOutputStream(OutputStream)} must be called before any value is written, and {@link #flush()} must + * be called before calling setBuffer or setOutputStream again to set another output. + */ + public interface LongEncodingWriter + { + /** + * Data will be written starting from current position of the buffer, and the position of the buffer will be + * updated as content is written. + */ + void setBuffer(ByteBuffer buffer); + + void setOutputStream(OutputStream output); + + void write(long value) throws IOException; + + /** + * Flush the unwritten content to the current output. + */ + void flush() throws IOException; + + /** + * Output the header values of the associating encoding format to the given outputStream. The header also include + * bytes for compression strategy and encoding format(optional) as described above in Compression Storage Format. + */ + void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException; + + /** + * Get the number of values that can be encoded into each block for the given block size in bytes + */ + int getBlockSize(int bytesPerBlock); + + /** + * Get the number of bytes required to encoding the given number of values + */ + int getNumBytes(int values); + } + + public interface LongEncodingReader + { + void setBuffer(ByteBuffer buffer); + + long read(int index); + + int getNumBytes(int values); + + LongEncodingReader duplicate(); + } + + public static Supplier getLongSupplier( + int totalSize, int sizePer, ByteBuffer fromBuffer, ByteOrder order, + LongEncodingFormat encodingFormat, + CompressedObjectStrategy.CompressionStrategy strategy + ) + { + if (strategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + return new EntireLayoutIndexedLongSupplier(totalSize, encodingFormat.getReader(fromBuffer, order)); + } else { + return new BlockLayoutIndexedLongSupplier(totalSize, sizePer, fromBuffer, order, + encodingFormat.getReader(fromBuffer, order), strategy + ); + } + } + + public static LongSupplierSerializer getLongSerializer( + IOPeon ioPeon, String filenameBase, ByteOrder order, + LongEncodingStrategy encodingStrategy, + CompressedObjectStrategy.CompressionStrategy compressionStrategy + ) + { + if (encodingStrategy == LongEncodingStrategy.AUTO) { + return new IntermediateLongSupplierSerializer(ioPeon, filenameBase, order, compressionStrategy); + } else if (encodingStrategy == LongEncodingStrategy.LONGS){ + if (compressionStrategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + return new EntireLayoutLongSupplierSerializer( + ioPeon, filenameBase, order, new LongsLongEncodingWriter(order) + ); + } else{ + return new BlockLayoutLongSupplierSerializer( + ioPeon, filenameBase, order, new LongsLongEncodingWriter(order), compressionStrategy + ); + } + } else { + throw new IAE("unknown encoding strategy : %s", encodingStrategy.toString()); + } + } + + // Float currently does not support any encoding types, and stores values as 4 byte float + + public static Supplier getFloatSupplier( + int totalSize, int sizePer, ByteBuffer fromBuffer, ByteOrder order, + CompressedObjectStrategy.CompressionStrategy strategy + ) + { + if (strategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + return new EntireLayoutIndexedFloatSupplier(totalSize, fromBuffer, order); + } else { + return new BlockLayoutIndexedFloatSupplier(totalSize, sizePer, fromBuffer, order, strategy); + } + } + + public static FloatSupplierSerializer getFloatSerializer( + IOPeon ioPeon, String filenameBase, ByteOrder order, + CompressedObjectStrategy.CompressionStrategy compressionStrategy + ) + { + if (compressionStrategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + return new EntireLayoutFloatSupplierSerializer( + ioPeon, filenameBase, order + ); + } else{ + return new BlockLayoutFloatSupplierSerializer( + ioPeon, filenameBase, order, compressionStrategy + ); + } + } + +} diff --git a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java new file mode 100644 index 000000000000..c31842979f66 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java @@ -0,0 +1,79 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.metamx.common.IAE; + +import java.nio.ByteBuffer; + +public class DeltaLongEncodingReader implements CompressionFactory.LongEncodingReader +{ + + private ByteBuffer buffer; + private final long base; + private final int bitsPerValue; + private VSizeLongSerde.LongDeserializer deserializer; + + public DeltaLongEncodingReader(ByteBuffer fromBuffer) + { + this.buffer = fromBuffer.asReadOnlyBuffer(); + byte version = buffer.get(); + if (version == CompressionFactory.DELTA_ENCODING_VERSION) { + base = buffer.getLong(); + bitsPerValue = buffer.getInt(); + fromBuffer.position(buffer.position()); + deserializer = VSizeLongSerde.getDeserializer(bitsPerValue, buffer, buffer.position()); + } else { + throw new IAE("Unknown version[%s]", version); + } + } + + private DeltaLongEncodingReader(ByteBuffer buffer, long base, int bitsPerValue) + { + this.buffer = buffer; + this.base = base; + this.bitsPerValue = bitsPerValue; + deserializer = VSizeLongSerde.getDeserializer(bitsPerValue, buffer, buffer.position()); + } + + @Override + public void setBuffer(ByteBuffer buffer) + { + deserializer = VSizeLongSerde.getDeserializer(bitsPerValue, buffer, buffer.position()); + } + + @Override + public long read(int index) + { + return base + deserializer.get(index); + } + + @Override + public int getNumBytes(int values) + { + return VSizeLongSerde.getSerializedSize(bitsPerValue, values); + } + + @Override + public CompressionFactory.LongEncodingReader duplicate() + { + return new DeltaLongEncodingReader(buffer.duplicate(), base, bitsPerValue); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java new file mode 100644 index 000000000000..922fa5e8fd47 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java @@ -0,0 +1,88 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +public class DeltaLongEncodingWriter implements CompressionFactory.LongEncodingWriter +{ + + private final long base; + private VSizeLongSerde.LongSerializer serializer; + private int bitsPerValue; + + public DeltaLongEncodingWriter(long base, long delta) + { + this.base = base; + this.bitsPerValue = VSizeLongSerde.getBitsForMax(delta + 1); + } + + @Override + public void setBuffer(ByteBuffer buffer) + { + serializer = VSizeLongSerde.getSerializer(bitsPerValue, buffer, buffer.position()); + } + + @Override + public void setOutputStream(OutputStream output) + { + serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); + } + + @Override + public void write(long value) throws IOException + { + serializer.write(value - base); + } + + public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException + { + metaOut.write(CompressionFactory.setEncodingFlag(strategy.getId())); + metaOut.write(CompressionFactory.LongEncodingFormat.DELTA.getId()); + metaOut.write(CompressionFactory.DELTA_ENCODING_VERSION); + metaOut.write(Longs.toByteArray(base)); + metaOut.write(Ints.toByteArray(bitsPerValue)); + } + + @Override + public int getBlockSize(int bytesPerBlock) + { + return VSizeLongSerde.getNumValuesPerBlock(bitsPerValue, bytesPerBlock); + } + + @Override + public int getNumBytes(int values) + { + return VSizeLongSerde.getSerializedSize(bitsPerValue, values); + } + + @Override + public void flush() throws IOException + { + if (serializer != null) { + serializer.close(); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java new file mode 100644 index 000000000000..8e0606c72257 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java @@ -0,0 +1,122 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.io.ByteSink; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; +import com.google.common.primitives.Floats; +import com.google.common.primitives.Ints; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; + +public class EntireLayoutFloatSupplierSerializer implements FloatSupplierSerializer +{ + private final IOPeon ioPeon; + private final String valueFile; + private final String metaFile; + private CountingOutputStream valuesOut; + private long metaCount = 0; + + private final ByteBuffer orderBuffer; + + private int numInserted = 0; + + public EntireLayoutFloatSupplierSerializer( + IOPeon ioPeon, String filenameBase, ByteOrder order + ) + { + this.ioPeon = ioPeon; + this.valueFile = filenameBase + ".value"; + this.metaFile = filenameBase + ".format"; + + orderBuffer = ByteBuffer.allocate(Floats.BYTES); + orderBuffer.order(order); + } + + @Override + public void open() throws IOException + { + valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + } + + @Override + public int size() + { + return numInserted; + } + + @Override + public void add(float value) throws IOException + { + orderBuffer.rewind(); + orderBuffer.putFloat(value); + valuesOut.write(orderBuffer.array()); + ++numInserted; + } + + @Override + public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + { + close(); + try (OutputStream out = consolidatedOut.openStream(); + InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream value = ioPeon.makeInputStream(valueFile)) { + ByteStreams.copy(meta, out); + ByteStreams.copy(value, out); + } + } + + @Override + public void close() throws IOException + { + valuesOut.close(); + try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { + metaOut.write(CompressedFloatsIndexedSupplier.version); + metaOut.write(Ints.toByteArray(numInserted)); + metaOut.write(Ints.toByteArray(0)); + metaOut.write(CompressedObjectStrategy.CompressionStrategy.NONE.getId()); + metaOut.close(); + metaCount = metaOut.getCount(); + } + } + + @Override + public long getSerializedSize() + { + return metaCount + valuesOut.getCount(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + try (InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream value = ioPeon.makeInputStream(valueFile)) { + ByteStreams.copy(Channels.newChannel(meta), channel); + ByteStreams.copy(Channels.newChannel(value), channel); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedFloatSupplier.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedFloatSupplier.java new file mode 100644 index 000000000000..57df3e95c32f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedFloatSupplier.java @@ -0,0 +1,89 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Supplier; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.FloatBuffer; + +public class EntireLayoutIndexedFloatSupplier implements Supplier +{ + private final int totalSize; + private FloatBuffer buffer; + + public EntireLayoutIndexedFloatSupplier(int totalSize, ByteBuffer fromBuffer, ByteOrder order) + { + this.totalSize = totalSize; + this.buffer = fromBuffer.asReadOnlyBuffer().order(order).asFloatBuffer(); + } + + @Override + public IndexedFloats get() + { + return new EntireLayoutIndexedFloats(); + } + + private class EntireLayoutIndexedFloats implements IndexedFloats + { + + @Override + public int size() + { + return totalSize; + } + + @Override + public float get(int index) + { + return buffer.get(buffer.position() + index); + } + + @Override + public void fill(int index, float[] toFill) + { + if (totalSize - index < toFill.length) { + throw new IndexOutOfBoundsException( + String.format( + "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize + ) + ); + } + for (int i = 0; i < toFill.length; i++) { + toFill[i] = get(index + i); + } + } + + @Override + public String toString() + { + return "EntireCompressedIndexedFloats_Anonymous{" + + ", totalSize=" + totalSize + + '}'; + } + + @Override + public void close() throws IOException + { + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedLongSupplier.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedLongSupplier.java new file mode 100644 index 000000000000..fe7c09fe6599 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutIndexedLongSupplier.java @@ -0,0 +1,87 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Supplier; + +import java.io.IOException; + +public class EntireLayoutIndexedLongSupplier implements Supplier +{ + + private final int totalSize; + private final CompressionFactory.LongEncodingReader reader; + + public EntireLayoutIndexedLongSupplier(int totalSize, CompressionFactory.LongEncodingReader reader) + { + this.totalSize = totalSize; + this.reader = reader; + } + + @Override + public IndexedLongs get() + { + return new EntireLayoutIndexedLongSupplier.EntireLayoutIndexedLongs(); + } + + private class EntireLayoutIndexedLongs implements IndexedLongs + { + + @Override + public int size() + { + return totalSize; + } + + @Override + public long get(int index) + { + return reader.read(index); + } + + @Override + public void fill(int index, long[] toFill) + { + if (totalSize - index < toFill.length) { + throw new IndexOutOfBoundsException( + String.format( + "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, totalSize + ) + ); + } + for (int i = 0; i < toFill.length; i++) { + toFill[i] = get(index + i); + } + } + + @Override + public String toString() + { + return "EntireCompressedIndexedLongs_Anonymous{" + + ", totalSize=" + totalSize + + '}'; + } + + @Override + public void close() throws IOException + { + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java new file mode 100644 index 000000000000..cb6b590ee73c --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java @@ -0,0 +1,119 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.io.ByteSink; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; +import com.google.common.primitives.Ints; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; + +public class EntireLayoutLongSupplierSerializer implements LongSupplierSerializer +{ + + private final IOPeon ioPeon; + private final String valueFile; + private final String metaFile; + private CountingOutputStream valuesOut; + private final CompressionFactory.LongEncodingWriter writer; + private long metaCount = 0; + + private int numInserted = 0; + + public EntireLayoutLongSupplierSerializer( + IOPeon ioPeon, String filenameBase, ByteOrder order, + CompressionFactory.LongEncodingWriter writer + ) + { + this.ioPeon = ioPeon; + this.valueFile = filenameBase + ".value"; + this.metaFile = filenameBase + ".format"; + this.writer = writer; + } + + @Override + public void open() throws IOException + { + valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + writer.setOutputStream(valuesOut); + } + + @Override + public int size() + { + return numInserted; + } + + @Override + public void add(long value) throws IOException + { + writer.write(value); + ++numInserted; + } + + @Override + public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + { + close(); + try (OutputStream out = consolidatedOut.openStream(); + InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream value = ioPeon.makeInputStream(valueFile)) { + ByteStreams.copy(meta, out); + ByteStreams.copy(value, out); + } + } + + @Override + public void close() throws IOException + { + writer.flush(); + valuesOut.close(); + try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { + metaOut.write(CompressedLongsIndexedSupplier.version); + metaOut.write(Ints.toByteArray(numInserted)); + metaOut.write(Ints.toByteArray(0)); + writer.putMeta(metaOut, CompressedObjectStrategy.CompressionStrategy.NONE); + metaOut.close(); + metaCount = metaOut.getCount(); + } + } + + @Override + public long getSerializedSize() + { + return metaCount + valuesOut.getCount(); + } + + @Override + public void writeToChannel(WritableByteChannel channel) throws IOException + { + try (InputStream meta = ioPeon.makeInputStream(metaFile); + InputStream value = ioPeon.makeInputStream(valueFile)) { + ByteStreams.copy(Channels.newChannel(meta), channel); + ByteStreams.copy(Channels.newChannel(value), channel); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/FloatSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/FloatSupplierSerializer.java new file mode 100644 index 000000000000..7a6620b923a2 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/FloatSupplierSerializer.java @@ -0,0 +1,36 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.io.ByteSink; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public interface FloatSupplierSerializer extends Closeable +{ + void open() throws IOException; + int size(); + void add(float value) throws IOException; + void closeAndConsolidate(ByteSink consolidatedOut) throws IOException; + long getSerializedSize(); + void writeToChannel(WritableByteChannel channel) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java index f8c42d7bfbd9..020fccd461f5 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -22,6 +22,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; +import com.google.common.io.ByteSource; import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; import com.google.common.io.InputSupplier; @@ -31,6 +32,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; @@ -132,20 +134,20 @@ public long getSerializedSize() valuesOut.getCount(); // value length } - public InputSupplier combineStreams() + public ByteSource combineStreams() { - return ByteStreams.join( + return ByteSource.concat( Iterables.transform( Arrays.asList("meta", "header", "values"), - new Function>() { + new Function() { @Override - public InputSupplier apply(final String input) + public ByteSource apply(final String input) { - return new InputSupplier() + return new ByteSource() { @Override - public InputStream getInput() throws IOException + public InputStream openStream() throws IOException { return ioPeon.makeInputStream(makeFilename(input)); } @@ -158,7 +160,7 @@ public InputStream getInput() throws IOException public void writeToChannel(WritableByteChannel channel) throws IOException { - final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput()); + final ReadableByteChannel from = Channels.newChannel(combineStreams().openStream()); ByteStreams.copy(from, channel); } } diff --git a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java deleted file mode 100644 index a970ed7e7d8f..000000000000 --- a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedFloats.java +++ /dev/null @@ -1,202 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.metamx.common.guava.CloseQuietly; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.FloatBuffer; -import java.util.Arrays; -import java.util.List; - -/** - */ -public class InMemoryCompressedFloats implements IndexedFloats -{ - public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; - private final CompressedFloatBufferObjectStrategy strategy; - private final int sizePer; - - private List compressedBuffers = Lists.newArrayList(); - private int numInserted = 0; - private int numCompressed = 0; - - private ResourceHolder holder = null; - private FloatBuffer loadBuffer = null; - private int loadBufferIndex = -1; - - private FloatBuffer endBuffer; - - public InMemoryCompressedFloats( - int sizePer, - ByteOrder order - ) - { - this.sizePer = sizePer; - strategy = CompressedFloatBufferObjectStrategy.getBufferForOrder( - order, - COMPRESSION, - sizePer - ); - - endBuffer = FloatBuffer.allocate(sizePer); - endBuffer.mark(); - } - - @Override - public int size() - { - return numInserted; - } - - public int add(float value) - { - int retVal = -1; - - if (! endBuffer.hasRemaining()) { - endBuffer.rewind(); - compressedBuffers.add(strategy.toBytes(StupidResourceHolder.create(endBuffer))); - endBuffer = FloatBuffer.allocate(sizePer); - endBuffer.mark(); - numCompressed += sizePer; - } - - retVal = numCompressed + endBuffer.position(); - endBuffer.put(value); - - ++numInserted; - return retVal; - } - - public int addAll(Iterable values) - { - int retVal = -1; - for (Float value : values) { - retVal = add(value); - } - return retVal; - } - - @Override - public float get(int index) - { - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - if (bufferNum == compressedBuffers.size()) { - return endBuffer.get(bufferIndex); - } - if (bufferNum != loadBufferIndex) { - loadBuffer(bufferNum); - } - - return loadBuffer.get(loadBuffer.position() + bufferIndex); - } - - @Override - public void fill(int index, float[] toFill) - { - if (size() - index < toFill.length) { - throw new IndexOutOfBoundsException( - String.format( - "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, size() - ) - ); - } - - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - int leftToFill = toFill.length; - while (leftToFill > 0) { - if (bufferNum == compressedBuffers.size()) { - endBuffer.mark(); - endBuffer.position(bufferIndex); - endBuffer.get(toFill, toFill.length - leftToFill, leftToFill); - endBuffer.rewind(); - return; - } - if (bufferNum != loadBufferIndex) { - loadBuffer(bufferNum); - } - - loadBuffer.mark(); - loadBuffer.position(loadBuffer.position() + bufferIndex); - final int numToGet = Math.min(loadBuffer.remaining(), leftToFill); - loadBuffer.get(toFill, toFill.length - leftToFill, numToGet); - loadBuffer.rewind(); - leftToFill -= numToGet; - ++bufferNum; - bufferIndex = 0; - } - } - - private void loadBuffer(int bufferNum) - { - loadBuffer = null; - CloseQuietly.close(holder); - final byte[] compressedBytes = compressedBuffers.get(bufferNum); - holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length); - loadBuffer = holder.get(); - loadBufferIndex = bufferNum; - } - - public CompressedFloatsIndexedSupplier toCompressedFloatsIndexedSupplier() - { - final FloatBuffer endBufCopy = endBuffer.asReadOnlyBuffer(); - endBufCopy.flip(); - - return new CompressedFloatsIndexedSupplier( - numInserted, - sizePer, - GenericIndexed.fromIterable( - Iterables.>concat( - Iterables.transform( - compressedBuffers, - new Function>() - { - @Override - public ResourceHolder apply(byte[] input) - { - return strategy.fromByteBuffer(ByteBuffer.wrap(input), input.length); - } - } - ), - Arrays.>asList(StupidResourceHolder.create(endBufCopy)) - ), - strategy - ), - COMPRESSION - ); - } - - @Override - public void close() throws IOException - { - CloseQuietly.close(holder); - } -} diff --git a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java b/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java deleted file mode 100644 index 6d14a1e1112e..000000000000 --- a/processing/src/main/java/io/druid/segment/data/InMemoryCompressedLongs.java +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.io.Closeables; -import com.metamx.common.guava.CloseQuietly; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; - -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.LongBuffer; -import java.util.Arrays; -import java.util.List; - -/** - */ -public class InMemoryCompressedLongs implements IndexedLongs -{ - public static final CompressedObjectStrategy.CompressionStrategy COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY; - private final CompressedLongBufferObjectStrategy strategy; - private final int sizePer; - - private List compressedBuffers = Lists.newArrayList(); - private int numInserted = 0; - private int numCompressed = 0; - - private ResourceHolder holder = null; - private LongBuffer loadBuffer = null; - private int loadBufferIndex = -1; - - private LongBuffer endBuffer; - - public InMemoryCompressedLongs( - int sizePer, - ByteOrder order - ) - { - this.sizePer = sizePer; - strategy = CompressedLongBufferObjectStrategy.getBufferForOrder( - order, - COMPRESSION, - sizePer - ); - - endBuffer = LongBuffer.allocate(sizePer); - endBuffer.mark(); - } - - @Override - public int size() - { - return numInserted; - } - - public int add(long value) - { - if (! endBuffer.hasRemaining()) { - endBuffer.rewind(); - compressedBuffers.add(strategy.toBytes(StupidResourceHolder.create(endBuffer))); - endBuffer = LongBuffer.allocate(sizePer); - endBuffer.mark(); - numCompressed += sizePer; - } - - int retVal = numCompressed + endBuffer.position(); - endBuffer.put(value); - - ++numInserted; - return retVal; - } - - public int addAll(Iterable values) - { - int retVal = -1; - for (Long value : values) { - retVal = add(value); - } - return retVal; - } - - @Override - public long get(int index) - { - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - if (bufferNum == compressedBuffers.size()) { - return endBuffer.get(bufferIndex); - } - if (bufferNum != loadBufferIndex) { - loadBuffer(bufferNum); - } - - return loadBuffer.get(loadBuffer.position() + bufferIndex); - } - - @Override - public void fill(int index, long[] toFill) - { - if (size() - index < toFill.length) { - throw new IndexOutOfBoundsException( - String.format( - "Cannot fill array of size[%,d] at index[%,d]. Max size[%,d]", toFill.length, index, size() - ) - ); - } - - int bufferNum = index / sizePer; - int bufferIndex = index % sizePer; - - int leftToFill = toFill.length; - while (leftToFill > 0) { - if (bufferNum == compressedBuffers.size()) { - endBuffer.mark(); - endBuffer.position(bufferIndex); - endBuffer.get(toFill, toFill.length - leftToFill, leftToFill); - endBuffer.rewind(); - return; - } - if (bufferNum != loadBufferIndex) { - loadBuffer(bufferNum); - } - - loadBuffer.mark(); - loadBuffer.position(loadBuffer.position() + bufferIndex); - final int numToGet = Math.min(loadBuffer.remaining(), leftToFill); - loadBuffer.get(toFill, toFill.length - leftToFill, numToGet); - loadBuffer.rewind(); - leftToFill -= numToGet; - ++bufferNum; - bufferIndex = 0; - } - } - - @Override - public int binarySearch(long key) - { - throw new UnsupportedOperationException(); - } - - @Override - public int binarySearch(long key, int from, int to) - { - throw new UnsupportedOperationException(); - } - - private void loadBuffer(int bufferNum) - { - loadBuffer = null; - CloseQuietly.close(holder); - final byte[] compressedBytes = compressedBuffers.get(bufferNum); - holder = strategy.fromByteBuffer(ByteBuffer.wrap(compressedBytes), compressedBytes.length); - loadBuffer = holder.get(); - loadBufferIndex = bufferNum; - } - - public CompressedLongsIndexedSupplier toCompressedLongsIndexedSupplier() - { - final LongBuffer longBufCopy = endBuffer.asReadOnlyBuffer(); - longBufCopy.flip(); - - return new CompressedLongsIndexedSupplier( - numInserted, - sizePer, - GenericIndexed.fromIterable( - Iterables.>concat( - Iterables.transform( - compressedBuffers, - new Function>() - { - @Override - public ResourceHolder apply(byte[] input) - { - return strategy.fromByteBuffer(ByteBuffer.wrap(input), input.length); - } - } - ), - Arrays.>asList(StupidResourceHolder.create(longBufCopy)) - ), - strategy - ), - COMPRESSION - ); - } - - @Override - public void close() throws IOException - { - Closeables.close(holder, false); - } -} diff --git a/processing/src/main/java/io/druid/segment/data/IndexedLongs.java b/processing/src/main/java/io/druid/segment/data/IndexedLongs.java index 9739502a64b7..8615178513e2 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedLongs.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedLongs.java @@ -29,6 +29,4 @@ public interface IndexedLongs extends Closeable public int size(); public long get(int index); public void fill(int index, long[] toFill); - int binarySearch(long key); - int binarySearch(long key, int from, int to); } diff --git a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java new file mode 100644 index 000000000000..076687bbb5d2 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java @@ -0,0 +1,150 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.collect.BiMap; +import com.google.common.collect.HashBiMap; +import com.google.common.io.ByteSink; +import com.google.common.io.CountingOutputStream; +import com.google.common.math.LongMath; +import com.google.common.primitives.Longs; + +import java.io.BufferedInputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.nio.ByteOrder; +import java.nio.channels.WritableByteChannel; + +public class IntermediateLongSupplierSerializer implements LongSupplierSerializer +{ + + private final IOPeon ioPeon; + private final String filenameBase; + private final String tempFile; + private final ByteOrder order; + private final CompressedObjectStrategy.CompressionStrategy compression; + private CountingOutputStream tempOut = null; + + private int numInserted = 0; + + private BiMap uniqueValues = HashBiMap.create(); + private long maxVal = Long.MIN_VALUE; + private long minVal = Long.MAX_VALUE; + + private LongSupplierSerializer delegate; + + public IntermediateLongSupplierSerializer( + IOPeon ioPeon, + String filenameBase, + ByteOrder order, + CompressedObjectStrategy.CompressionStrategy compression + ) + { + this.ioPeon = ioPeon; + this.tempFile = filenameBase + ".temp"; + this.filenameBase = filenameBase; + this.order = order; + this.compression = compression; + } + + public void open() throws IOException + { + tempOut = new CountingOutputStream(ioPeon.makeOutputStream(tempFile)); + } + + public int size() + { + return numInserted; + } + + public void add(long value) throws IOException + { + tempOut.write(Longs.toByteArray(value)); + ++numInserted; + if (uniqueValues.size() <= CompressionFactory.MAX_TABLE_SIZE && !uniqueValues.containsKey(value)) { + uniqueValues.put(value, uniqueValues.size()); + } + if (value > maxVal) { + maxVal = value; + } + if (value < minVal) { + minVal = value; + } + } + + private void makeDelegate() throws IOException + { + CompressionFactory.LongEncodingWriter writer; + long delta; + try { + delta = LongMath.checkedSubtract(maxVal, minVal); + } + catch (ArithmeticException e) { + delta = -1; + } + if (uniqueValues.size() <= CompressionFactory.MAX_TABLE_SIZE) { + writer = new TableLongEncodingWriter(uniqueValues); + } else if (delta != -1 && delta != Long.MAX_VALUE) { + writer = new DeltaLongEncodingWriter(minVal, delta); + } else { + writer = new LongsLongEncodingWriter(order); + } + + if (compression == CompressedObjectStrategy.CompressionStrategy.NONE) { + delegate = new EntireLayoutLongSupplierSerializer( + ioPeon, filenameBase, order, writer + ); + } else { + delegate = new BlockLayoutLongSupplierSerializer( + ioPeon, filenameBase, order, writer, compression + ); + } + + DataInputStream tempIn = new DataInputStream(new BufferedInputStream(ioPeon.makeInputStream(tempFile))); + delegate.open(); + while (tempIn.available() > 0) { + delegate.add(tempIn.readLong()); + } + } + + public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + { + tempOut.close(); + makeDelegate(); + delegate.closeAndConsolidate(consolidatedOut); + } + + public void close() throws IOException + { + tempOut.close(); + makeDelegate(); + delegate.close(); + } + + public long getSerializedSize() + { + return delegate.getSerializedSize(); + } + + public void writeToChannel(WritableByteChannel channel) throws IOException + { + delegate.writeToChannel(channel); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java new file mode 100644 index 000000000000..0356baba6e5a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java @@ -0,0 +1,38 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.io.ByteSink; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +/** + */ +public interface LongSupplierSerializer extends Closeable +{ + void open() throws IOException; + int size(); + void add(long value) throws IOException; + void closeAndConsolidate(ByteSink consolidatedOut) throws IOException; + long getSerializedSize(); + void writeToChannel(WritableByteChannel channel) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/data/LongsLongEncodingReader.java b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingReader.java new file mode 100644 index 000000000000..9cbb4f96a70b --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingReader.java @@ -0,0 +1,65 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.primitives.Longs; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.LongBuffer; + +public class LongsLongEncodingReader implements CompressionFactory.LongEncodingReader +{ + private LongBuffer buffer; + + public LongsLongEncodingReader(ByteBuffer fromBuffer, ByteOrder order) + { + this.buffer = fromBuffer.asReadOnlyBuffer().order(order).asLongBuffer(); + } + + private LongsLongEncodingReader(LongBuffer buffer) + { + this.buffer = buffer; + } + + @Override + public void setBuffer(ByteBuffer buffer) + { + this.buffer = buffer.asLongBuffer(); + } + + @Override + public long read(int index) + { + return buffer.get(buffer.position() + index); + } + + @Override + public int getNumBytes(int values) + { + return values * Longs.BYTES; + } + + @Override + public CompressionFactory.LongEncodingReader duplicate() + { + return new LongsLongEncodingReader(buffer.duplicate()); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java new file mode 100644 index 000000000000..8a32d01e999f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java @@ -0,0 +1,96 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.primitives.Longs; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class LongsLongEncodingWriter implements CompressionFactory.LongEncodingWriter +{ + + private final ByteBuffer orderBuffer; + private final ByteOrder order; + private ByteBuffer outBuffer = null; + private OutputStream outStream = null; + + public LongsLongEncodingWriter(ByteOrder order) + { + this.order = order; + orderBuffer = ByteBuffer.allocate(Longs.BYTES); + orderBuffer.order(order); + } + + @Override + public void setBuffer(ByteBuffer buffer) + { + outStream = null; + outBuffer = buffer; + // this order change is safe as the buffer is passed in and allocated in BlockLayoutLongSupplierSerializer, and + // is used only as a temporary storage to be written + outBuffer.order(order); + } + + @Override + public void setOutputStream(OutputStream output) + { + outBuffer = null; + outStream = output; + } + + @Override + public void write(long value) throws IOException + { + if (outBuffer != null) { + outBuffer.putLong(value); + } + if (outStream != null) { + orderBuffer.rewind(); + orderBuffer.putLong(value); + outStream.write(orderBuffer.array()); + } + } + + @Override + public void flush() throws IOException + { + } + + @Override + public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException + { + metaOut.write(strategy.getId()); + } + + @Override + public int getBlockSize(int bytesPerBlock) + { + return bytesPerBlock / Longs.BYTES; + } + + @Override + public int getNumBytes(int values) + { + return values * Longs.BYTES; + } +} diff --git a/processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java b/processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java new file mode 100644 index 000000000000..a1852d71a55f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java @@ -0,0 +1,85 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.metamx.common.IAE; + +import java.nio.ByteBuffer; + +public class TableLongEncodingReader implements CompressionFactory.LongEncodingReader +{ + private final long table[]; + private final int bitsPerValue; + private final ByteBuffer buffer; + private VSizeLongSerde.LongDeserializer deserializer; + + public TableLongEncodingReader(ByteBuffer fromBuffer) + { + this.buffer = fromBuffer.asReadOnlyBuffer(); + byte version = buffer.get(); + if (version == CompressionFactory.TABLE_ENCODING_VERSION) { + int tableSize = buffer.getInt(); + if (tableSize < 0 || tableSize > CompressionFactory.MAX_TABLE_SIZE) { + throw new IAE("Invalid table size[%s]", tableSize); + } + bitsPerValue = VSizeLongSerde.getBitsForMax(tableSize); + table = new long[tableSize]; + for (int i = 0; i < tableSize; i++) { + table[i] = buffer.getLong(); + } + fromBuffer.position(buffer.position()); + deserializer = VSizeLongSerde.getDeserializer(bitsPerValue, buffer, buffer.position()); + } else { + throw new IAE("Unknown version[%s]", version); + } + } + + private TableLongEncodingReader(ByteBuffer buffer, long table[], int bitsPerValue) + { + this.buffer = buffer; + this.table = table; + this.bitsPerValue = bitsPerValue; + deserializer = VSizeLongSerde.getDeserializer(bitsPerValue, buffer, buffer.position()); + } + + @Override + public void setBuffer(ByteBuffer buffer) + { + deserializer = VSizeLongSerde.getDeserializer(bitsPerValue, buffer, buffer.position()); + } + + @Override + public long read(int index) + { + return table[(int) deserializer.get(index)]; + } + + @Override + public int getNumBytes(int values) + { + return VSizeLongSerde.getSerializedSize(bitsPerValue, values); + } + + @Override + public CompressionFactory.LongEncodingReader duplicate() + { + return new TableLongEncodingReader(buffer.duplicate(), table, bitsPerValue); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java new file mode 100644 index 000000000000..2e2d8b43cd93 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java @@ -0,0 +1,97 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.collect.BiMap; +import com.google.common.primitives.Ints; +import com.google.common.primitives.Longs; +import com.metamx.common.IAE; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +public class TableLongEncodingWriter implements CompressionFactory.LongEncodingWriter +{ + + private final BiMap table; + private final int bitsPerValue; + private VSizeLongSerde.LongSerializer serializer; + + public TableLongEncodingWriter(BiMap table) + { + if (table.size() > CompressionFactory.MAX_TABLE_SIZE) { + throw new IAE("Invalid table size[%s]", table.size()); + } + this.table = table; + this.bitsPerValue = VSizeLongSerde.getBitsForMax(table.size()); + } + + @Override + public void setBuffer(ByteBuffer buffer) + { + serializer = VSizeLongSerde.getSerializer(bitsPerValue, buffer, buffer.position()); + } + + @Override + public void setOutputStream(OutputStream output) + { + serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); + } + + @Override + public void write(long value) throws IOException + { + serializer.write(table.get(value)); + } + + @Override + public void flush() throws IOException + { + if (serializer != null) { + serializer.close(); + } + } + + @Override + public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException + { + metaOut.write(CompressionFactory.setEncodingFlag(strategy.getId())); + metaOut.write(CompressionFactory.LongEncodingFormat.TABLE.getId()); + metaOut.write(CompressionFactory.TABLE_ENCODING_VERSION); + metaOut.write(Ints.toByteArray(table.size())); + BiMap inverse = table.inverse(); + for (int i = 0; i < table.size(); i++) { + metaOut.write(Longs.toByteArray(inverse.get(i))); + } + } + + @Override + public int getBlockSize(int bytesPerBlock) + { + return VSizeLongSerde.getNumValuesPerBlock(bitsPerValue, bytesPerBlock); + } + + @Override + public int getNumBytes(int values) + { + return VSizeLongSerde.getSerializedSize(bitsPerValue, values); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java new file mode 100644 index 000000000000..e082579d3503 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java @@ -0,0 +1,86 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class VSizeCompressedObjectStrategy extends CompressedObjectStrategy +{ + + private final int expectedBytes; + + public static VSizeCompressedObjectStrategy getBufferForOrder( + final ByteOrder order, + final CompressionStrategy compression, + final int expectedBytes + ) + { + return new VSizeCompressedObjectStrategy(order, compression, expectedBytes); + } + + protected VSizeCompressedObjectStrategy( + ByteOrder order, + CompressionStrategy compression, + int expectedBytes + ) + { + super(order, new BufferConverter() + { + @Override + public ByteBuffer convert(ByteBuffer buf) + { + return buf; + } + + @Override + public int compare(ByteBuffer lhs, ByteBuffer rhs) + { + return CompressedByteBufferObjectStrategy.ORDERING.compare(lhs, rhs); + } + + @Override + public int sizeOf(int count) + { + return count; // 1 byte per element + } + + @Override + public ByteBuffer combine(ByteBuffer into, ByteBuffer from) + { + return into.put(from); + } + }, compression); + + this.expectedBytes = expectedBytes; + } + + @Override + protected ByteBuffer bufferFor(ByteBuffer val) + { + return ByteBuffer.allocate(expectedBytes).order(order); + } + + @Override + protected void decompress(ByteBuffer buffer, int numBytes, ByteBuffer buf) + { + decompressor.decompress(buffer, numBytes, buf, expectedBytes); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java b/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java new file mode 100644 index 000000000000..20221bac5e8a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java @@ -0,0 +1,634 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.metamx.common.IAE; + +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * Currently only support big endian + *

+ * An empty 4 bytes is written upon closing to avoid index out of bound exception for deserializers that shift bytes + */ +public class VSizeLongSerde +{ + + public static final int SUPPORTED_SIZE[] = {1, 2, 4, 8, 12, 16, 20, 24, 32, 40, 48, 56, 64}; + public static final byte EMPTY[] = {0, 0, 0, 0}; + + public static int getBitsForMax(long value) + { + if (value < 0) { + throw new IAE("maxValue[%s] must be positive", value); + } + byte numBits = 0; + long maxValue = 1; + for (int i = 0; i < SUPPORTED_SIZE.length; i++) { + while (numBits < SUPPORTED_SIZE[i] && maxValue < Long.MAX_VALUE / 2) { + numBits++; + maxValue *= 2; + } + if (value <= maxValue || maxValue >= Long.MAX_VALUE / 2) { + return SUPPORTED_SIZE[i]; + } + } + return 64; + } + + public static int getSerializedSize(int bitsPerValue, int numValues) + { + // this value is calculated by rounding up the byte and adding the 4 closing bytes + return (bitsPerValue * numValues + 7) / 8 + 4; + } + + // block size should be power of 2 so get of indexedLong can be optimized using bit operators + public static int getNumValuesPerBlock(int bitsPerValue, int blockSize) + { + int ret = 1; + while (getSerializedSize(bitsPerValue, ret) <= blockSize) { + ret *= 2; + } + return ret / 2; + } + + public static LongSerializer getSerializer(int longSize, OutputStream output) + { + switch (longSize) { + case 1: + return new Size1Ser(output); + case 2: + return new Size2Ser(output); + case 4: + return new Mult4Ser(output, 0); + case 8: + return new Mult8Ser(output, 1); + case 12: + return new Mult4Ser(output, 1); + case 16: + return new Mult8Ser(output, 2); + case 20: + return new Mult4Ser(output, 2); + case 24: + return new Mult8Ser(output, 3); + case 32: + return new Mult8Ser(output, 4); + case 40: + return new Mult8Ser(output, 5); + case 48: + return new Mult8Ser(output, 6); + case 56: + return new Mult8Ser(output, 7); + case 64: + return new Mult8Ser(output, 8); + default: + throw new IAE("Unsupported size %s", longSize); + } + } + + public static LongSerializer getSerializer(int longSize, ByteBuffer buffer, int bufferOffset) + { + switch (longSize) { + case 1: + return new Size1Ser(buffer, bufferOffset); + case 2: + return new Size2Ser(buffer, bufferOffset); + case 4: + return new Mult4Ser(buffer, bufferOffset, 0); + case 8: + return new Mult8Ser(buffer, bufferOffset, 1); + case 12: + return new Mult4Ser(buffer, bufferOffset, 1); + case 16: + return new Mult8Ser(buffer, bufferOffset, 2); + case 20: + return new Mult4Ser(buffer, bufferOffset, 2); + case 24: + return new Mult8Ser(buffer, bufferOffset, 3); + case 32: + return new Mult8Ser(buffer, bufferOffset, 4); + case 40: + return new Mult8Ser(buffer, bufferOffset, 5); + case 48: + return new Mult8Ser(buffer, bufferOffset, 6); + case 56: + return new Mult8Ser(buffer, bufferOffset, 7); + case 64: + return new Mult8Ser(buffer, bufferOffset, 8); + default: + throw new IAE("Unsupported size %s", longSize); + } + } + + + // LongDeserializers were adapted from Apache Lucene DirectReader, see: + // https://github.com/apache/lucene-solr/blob/master/lucene/core/src/java/org/apache/lucene/util/packed/DirectReader.java + public static LongDeserializer getDeserializer(int longSize, ByteBuffer fromBuffer, int bufferOffset) + { + // The buffer needs to be duplicated since the byte order is changed + ByteBuffer buffer = fromBuffer.duplicate().order(ByteOrder.BIG_ENDIAN); + switch (longSize) { + case 1: + return new Size1Des(buffer, bufferOffset); + case 2: + return new Size2Des(buffer, bufferOffset); + case 4: + return new Size4Des(buffer, bufferOffset); + case 8: + return new Size8Des(buffer, bufferOffset); + case 12: + return new Size12Des(buffer, bufferOffset); + case 16: + return new Size16Des(buffer, bufferOffset); + case 20: + return new Size20Des(buffer, bufferOffset); + case 24: + return new Size24Des(buffer, bufferOffset); + case 32: + return new Size32Des(buffer, bufferOffset); + case 40: + return new Size40Des(buffer, bufferOffset); + case 48: + return new Size48Des(buffer, bufferOffset); + case 56: + return new Size56Des(buffer, bufferOffset); + case 64: + return new Size64Des(buffer, bufferOffset); + default: + throw new IAE("Unsupported size %s", longSize); + } + } + + public interface LongSerializer extends Closeable + { + void write(long value) throws IOException; + } + + private static final class Size1Ser implements LongSerializer + { + OutputStream output = null; + ByteBuffer buffer; + byte curByte = 0; + int count = 0; + + public Size1Ser(OutputStream output) + { + this.output = output; + this.buffer = ByteBuffer.allocate(1); + } + + public Size1Ser(ByteBuffer buffer, int offset) + { + this.buffer = buffer; + this.buffer.position(offset); + } + + @Override + public void write(long value) throws IOException + { + if (count == 8) { + buffer.put(curByte); + count = 0; + if (!buffer.hasRemaining() && output != null) { + output.write(buffer.array()); + buffer.rewind(); + } + } + curByte = (byte) ((curByte << 1) | (value & 1)); + count++; + } + + @Override + public void close() throws IOException + { + buffer.put((byte) (curByte << (8 - count))); + if (output != null) { + output.write(buffer.array()); + output.write(EMPTY); + output.flush(); + } else { + buffer.putInt(0); + } + } + } + + private static final class Size2Ser implements LongSerializer + { + OutputStream output = null; + ByteBuffer buffer; + byte curByte = 0; + int count = 0; + + public Size2Ser(OutputStream output) + { + this.output = output; + this.buffer = ByteBuffer.allocate(1); + } + + public Size2Ser(ByteBuffer buffer, int offset) + { + this.buffer = buffer; + this.buffer.position(offset); + } + + @Override + public void write(long value) throws IOException + { + if (count == 8) { + buffer.put(curByte); + count = 0; + if (!buffer.hasRemaining() && output != null) { + output.write(buffer.array()); + buffer.rewind(); + } + } + curByte = (byte) ((curByte << 2) | (value & 3)); + count += 2; + } + + @Override + public void close() throws IOException + { + buffer.put((byte) (curByte << (8 - count))); + if (output != null) { + output.write(buffer.array()); + output.write(EMPTY); + output.flush(); + } else { + buffer.putInt(0); + } + } + } + + private static final class Mult4Ser implements LongSerializer + { + + OutputStream output = null; + ByteBuffer buffer; + int numBytes; + byte curByte = 0; + boolean first = true; + + public Mult4Ser(OutputStream output, int numBytes) + { + this.output = output; + this.buffer = ByteBuffer.allocate(numBytes * 2 + 1); + this.numBytes = numBytes; + } + + public Mult4Ser(ByteBuffer buffer, int offset, int numBytes) + { + this.buffer = buffer; + this.buffer.position(offset); + this.numBytes = numBytes; + } + + @Override + public void write(long value) throws IOException + { + int shift = 0; + if (first) { + shift = 4; + curByte = (byte) value; + first = false; + } else { + curByte = (byte) ((curByte << 4) | ((value >> (numBytes << 3)) & 0xF)); + buffer.put(curByte); + first = true; + } + for (int i = numBytes - 1; i >= 0; i--) { + buffer.put((byte) (value >>> (i * 8 + shift))); + } + if (!buffer.hasRemaining() && output != null) { + output.write(buffer.array()); + buffer.rewind(); + } + } + + @Override + public void close() throws IOException + { + if (!first) { + buffer.put((byte) (curByte << 4)); + } + if (output != null) { + output.write(buffer.array(), 0, buffer.position()); + output.write(EMPTY); + output.flush(); + } else { + buffer.putInt(0); + } + } + } + + private static final class Mult8Ser implements LongSerializer + { + OutputStream output; + ByteBuffer buffer; + int numBytes; + + public Mult8Ser(OutputStream output, int numBytes) + { + this.output = output; + this.buffer = ByteBuffer.allocate(1); + this.numBytes = numBytes; + } + + public Mult8Ser(ByteBuffer buffer, int offset, int numBytes) + { + this.buffer = buffer; + this.buffer.position(offset); + this.numBytes = numBytes; + } + + @Override + public void write(long value) throws IOException + { + for (int i = numBytes - 1; i >= 0; i--) { + buffer.put((byte) (value >>> (i * 8))); + if (output != null) { + output.write(buffer.array()); + buffer.position(0); + } + } + } + + @Override + public void close() throws IOException + { + if (output != null) { + output.write(EMPTY); + output.flush(); + } else { + buffer.putInt(0); + } + } + } + + public interface LongDeserializer + { + long get(int index); + } + + private static final class Size1Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size1Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + int shift = 7 - (index & 7); + return (buffer.get(offset + (index >> 3)) >> shift) & 1; + } + } + + private static final class Size2Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size2Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + int shift = 6 - ((index & 3) << 1); + return (buffer.get(offset + (index >> 2)) >> shift) & 3; + } + } + + private static final class Size4Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size4Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + int shift = ((index + 1) & 1) << 2; + return (buffer.get(offset + (index >> 1)) >> shift) & 0xF; + } + } + + private static final class Size8Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size8Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + return buffer.get(offset + index) & 0xFF; + } + } + + private static final class Size12Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size12Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + int shift = ((index + 1) & 1) << 2; + int offset = (index * 3) >> 1; + return (buffer.getShort(this.offset + offset) >> shift) & 0xFFF; + } + } + + private static final class Size16Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size16Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + public long get(int index) + { + return buffer.getShort(offset + (index << 1)) & 0xFFFF; + } + } + + private static final class Size20Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size20Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + int shift = (((index + 1) & 1) << 2) + 8; + int offset = (index * 5) >> 1; + return (buffer.getInt(this.offset + offset) >> shift) & 0xFFFFF; + } + } + + private static final class Size24Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size24Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + return buffer.getInt(offset + index * 3) >>> 8; + } + } + + private static final class Size32Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size32Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + return buffer.getInt(offset + (index << 2)) & 0xFFFFFFFFL; + } + } + + private static final class Size40Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size40Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + return buffer.getLong(offset + index * 5) >>> 24; + } + } + + private static final class Size48Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size48Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + return buffer.getLong(offset + index * 6) >>> 16; + } + } + + private static final class Size56Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size56Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + return buffer.getLong(offset + index * 7) >>> 8; + } + } + + private static final class Size64Des implements LongDeserializer + { + final ByteBuffer buffer; + final int offset; + + public Size64Des(ByteBuffer buffer, int bufferOffset) + { + this.buffer = buffer; + this.offset = bufferOffset; + } + + @Override + public long get(int index) + { + return buffer.getLong(offset + (index << 3)); + } + } + +} diff --git a/processing/src/test/java/io/druid/segment/IndexIOTest.java b/processing/src/test/java/io/druid/segment/IndexIOTest.java index 85d61cc9c913..2c79972a5fab 100644 --- a/processing/src/test/java/io/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/io/druid/segment/IndexIOTest.java @@ -35,6 +35,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; @@ -68,7 +69,8 @@ public class IndexIOTest private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( new ConciseBitmapSerdeFactory(), CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4 + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionFactory.LongEncodingStrategy.LONGS ); private static List filterByBitset(List list, BitSet bitSet) diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTest.java b/processing/src/test/java/io/druid/segment/IndexMergerTest.java index cd7a3fe24ffc..ca7861457639 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTest.java @@ -41,6 +41,7 @@ import io.druid.segment.column.SimpleDictionaryEncodedColumn; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.data.IndexedInts; @@ -68,6 +69,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.EnumSet; import java.util.List; import java.util.Map; @@ -80,7 +82,7 @@ public class IndexMergerTest protected IndexMerger INDEX_MERGER; private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); - @Parameterized.Parameters(name = "{index}: useV9={0}, bitmap={1}, metric compression={2}, dimension compression={3}") + @Parameterized.Parameters(name = "{index}: useV9={0}, bitmap={1}, metric compression={2}, dimension compression={3}, long encoding={4}") public static Collection data() { return Collections2.transform( @@ -94,16 +96,9 @@ public static Collection data() new RoaringBitmapSerdeFactory(null), new ConciseBitmapSerdeFactory() ), - ImmutableSet.of( - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZF - ), - ImmutableSet.of( - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZF - ) + EnumSet.allOf(CompressedObjectStrategy.CompressionStrategy.class), + ImmutableSet.copyOf(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), + EnumSet.allOf(CompressionFactory.LongEncodingStrategy.class) ) ), new Function, Object[]>() { @@ -120,14 +115,16 @@ public Object[] apply(List input) static IndexSpec makeIndexSpec( BitmapSerdeFactory bitmapSerdeFactory, CompressedObjectStrategy.CompressionStrategy compressionStrategy, - CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy + CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy, + CompressionFactory.LongEncodingStrategy longEncodingStrategy ) { if (bitmapSerdeFactory != null || compressionStrategy != null) { return new IndexSpec( bitmapSerdeFactory, - compressionStrategy.name().toLowerCase(), - dimCompressionStrategy.name().toLowerCase() + dimCompressionStrategy, + compressionStrategy, + longEncodingStrategy ); } else { return new IndexSpec(); @@ -142,10 +139,11 @@ public IndexMergerTest( boolean useV9, BitmapSerdeFactory bitmapSerdeFactory, CompressedObjectStrategy.CompressionStrategy compressionStrategy, - CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy + CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy, + CompressionFactory.LongEncodingStrategy longEncodingStrategy ) { - this.indexSpec = makeIndexSpec(bitmapSerdeFactory, compressionStrategy, dimCompressionStrategy); + this.indexSpec = makeIndexSpec(bitmapSerdeFactory, compressionStrategy, dimCompressionStrategy, longEncodingStrategy); if (useV9) { INDEX_MERGER = TestHelper.getTestIndexMergerV9(); } else { @@ -176,7 +174,7 @@ public void testPersist() throws Exception Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); Assert.assertEquals(3, index.getColumnNames().size()); - assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index, indexSpec.getDimensionCompression()); Assert.assertArrayEquals( IncrementalIndexTest.getDefaultCombiningAggregatorFactories(), @@ -222,7 +220,7 @@ public void testPersistWithDifferentDims() throws Exception Assert.assertEquals(2, index.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); Assert.assertEquals(3, index.getColumnNames().size()); - assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index, indexSpec.getDimensionCompression()); final QueryableIndexIndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); final List boatList = ImmutableList.copyOf(adapter.getRows()); @@ -265,7 +263,7 @@ public void testPersistWithSegmentMetadata() throws Exception Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index.getAvailableDimensions())); Assert.assertEquals(3, index.getColumnNames().size()); - assertDimCompression(index, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index, indexSpec.getDimensionCompression()); Assert.assertEquals( new Metadata() @@ -359,9 +357,9 @@ public void testPersistMerge() throws Exception Assert.assertEquals(3, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(3, merged.getColumnNames().size()); - assertDimCompression(index2, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index2, indexSpec.getDimensionCompression()); + assertDimCompression(index1, indexSpec.getDimensionCompression()); + assertDimCompression(merged, indexSpec.getDimensionCompression()); Assert.assertArrayEquals( getCombiningAggregators(mergedAggregators), @@ -443,9 +441,9 @@ public void testPersistEmptyColumn() throws Exception Assert.assertEquals(2, merged.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(ImmutableList.of("dim2"), ImmutableList.copyOf(merged.getAvailableDimensions())); - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(index2, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index1, indexSpec.getDimensionCompression()); + assertDimCompression(index2, indexSpec.getDimensionCompression()); + assertDimCompression(merged, indexSpec.getDimensionCompression()); } @Test @@ -502,8 +500,8 @@ public void testMergeRetainsValues() throws Exception INDEX_IO.validateTwoSegments(tempDir1, mergedDir); - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index1, indexSpec.getDimensionCompression()); + assertDimCompression(merged, indexSpec.getDimensionCompression()); } @Test @@ -561,8 +559,8 @@ public void testAppendRetainsValues() throws Exception INDEX_IO.validateTwoSegments(tempDir1, mergedDir); - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(merged, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index1, indexSpec.getDimensionCompression()); + assertDimCompression(merged, indexSpec.getDimensionCompression()); Assert.assertArrayEquals( getCombiningAggregators(mergedAggregators), @@ -607,8 +605,15 @@ public void testMergeSpecChange() throws Exception IndexSpec newSpec = new IndexSpec( indexSpec.getBitmapSerdeFactory(), - "lz4".equals(indexSpec.getDimensionCompression()) ? "lzf" : "lz4", - "lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4" + CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressedObjectStrategy.CompressionStrategy.LZF : + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressedObjectStrategy.CompressionStrategy.LZF : + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionFactory.LongEncodingStrategy.LONGS.equals(indexSpec.getLongEncoding()) ? + CompressionFactory.LongEncodingStrategy.AUTO : + CompressionFactory.LongEncodingStrategy.LONGS ); AggregatorFactory[] mergedAggregators = new AggregatorFactory[]{new CountAggregatorFactory("count")}; @@ -630,8 +635,8 @@ public void testMergeSpecChange() throws Exception INDEX_IO.validateTwoSegments(tempDir1, mergedDir); - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(merged, newSpec.getDimensionCompressionStrategy()); + assertDimCompression(index1, indexSpec.getDimensionCompression()); + assertDimCompression(merged, newSpec.getDimensionCompression()); } @@ -688,8 +693,8 @@ public void testConvertSame() throws Exception INDEX_IO.validateTwoSegments(tempDir1, convertDir); - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(converted, indexSpec.getDimensionCompressionStrategy()); + assertDimCompression(index1, indexSpec.getDimensionCompression()); + assertDimCompression(converted, indexSpec.getDimensionCompression()); Assert.assertArrayEquals( getCombiningAggregators(aggregators), @@ -744,8 +749,15 @@ public void testConvertDifferent() throws Exception IndexSpec newSpec = new IndexSpec( indexSpec.getBitmapSerdeFactory(), - "lz4".equals(indexSpec.getDimensionCompression()) ? "lzf" : "lz4", - "lz4".equals(indexSpec.getMetricCompression()) ? "lzf" : "lz4" + CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressedObjectStrategy.CompressionStrategy.LZF : + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressedObjectStrategy.CompressionStrategy.LZF : + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionFactory.LongEncodingStrategy.LONGS.equals(indexSpec.getLongEncoding()) ? + CompressionFactory.LongEncodingStrategy.AUTO : + CompressionFactory.LongEncodingStrategy.LONGS ); QueryableIndex converted = closer.closeLater( @@ -764,8 +776,8 @@ public void testConvertDifferent() throws Exception INDEX_IO.validateTwoSegments(tempDir1, convertDir); - assertDimCompression(index1, indexSpec.getDimensionCompressionStrategy()); - assertDimCompression(converted, newSpec.getDimensionCompressionStrategy()); + assertDimCompression(index1, indexSpec.getDimensionCompression()); + assertDimCompression(converted, newSpec.getDimensionCompression()); Assert.assertArrayEquals( getCombiningAggregators(aggregators), diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java index ea988ac54a53..34cffe1bb100 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -32,6 +32,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.OnheapIncrementalIndex; @@ -75,7 +76,8 @@ public class IndexMergerV9CompatibilityTest private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( new ConciseBitmapSerdeFactory(), CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4 + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionFactory.LongEncodingStrategy.LONGS ); private static final List DIMS = ImmutableList.of("dim0", "dim1"); diff --git a/processing/src/test/java/io/druid/segment/IndexSpecTest.java b/processing/src/test/java/io/druid/segment/IndexSpecTest.java index 2053c69869a1..68223b5680c2 100644 --- a/processing/src/test/java/io/druid/segment/IndexSpecTest.java +++ b/processing/src/test/java/io/druid/segment/IndexSpecTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.RoaringBitmapSerdeFactory; import org.junit.Assert; import org.junit.Test; @@ -32,12 +33,14 @@ public class IndexSpecTest public void testSerde() throws Exception { final ObjectMapper objectMapper = new DefaultObjectMapper(); - final String json = "{ \"bitmap\" : { \"type\" : \"roaring\" }, \"dimensionCompression\" : \"lz4\", \"metricCompression\" : \"lzf\" }"; + final String json = "{ \"bitmap\" : { \"type\" : \"roaring\" }, \"dimensionCompression\" : \"lz4\", \"metricCompression\" : \"lzf\"" + + ", \"longEncoding\" : \"auto\" }"; final IndexSpec spec = objectMapper.readValue(json, IndexSpec.class); Assert.assertEquals(new RoaringBitmapSerdeFactory(null), spec.getBitmapSerdeFactory()); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompressionStrategy()); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZF, spec.getMetricCompressionStrategy()); + Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompression()); + Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZF, spec.getMetricCompression()); + Assert.assertEquals(CompressionFactory.LongEncodingStrategy.AUTO, spec.getLongEncoding()); Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class)); } @@ -50,8 +53,7 @@ public void testSerdeUncompressed() throws Exception final IndexSpec spec = objectMapper.readValue(json, IndexSpec.class); - Assert.assertEquals(IndexSpec.UNCOMPRESSED, spec.getDimensionCompression()); - Assert.assertEquals(null, spec.getDimensionCompressionStrategy()); + Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, spec.getDimensionCompression()); Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class)); } @@ -59,7 +61,8 @@ public void testSerdeUncompressed() throws Exception public void testDefaults() throws Exception { final IndexSpec spec = new IndexSpec(); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompressionStrategy()); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getMetricCompressionStrategy()); + Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompression()); + Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getMetricCompression()); + Assert.assertEquals(CompressionFactory.LongEncodingStrategy.LONGS, spec.getLongEncoding()); } } diff --git a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java index 0d3e98181a25..e3a02fabac2e 100644 --- a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java @@ -20,6 +20,7 @@ package io.druid.segment; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.data.IndexedInts; @@ -38,7 +39,8 @@ public class QueryableIndexIndexableAdapterTest private static final IndexSpec INDEX_SPEC = IndexMergerTest.makeIndexSpec( new ConciseBitmapSerdeFactory(), CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4 + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionFactory.LongEncodingStrategy.LONGS ); @Rule diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java deleted file mode 100644 index 23e375f7728b..000000000000 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsIndexedSupplierTest.java +++ /dev/null @@ -1,351 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import com.google.common.io.Closeables; -import com.google.common.primitives.Floats; -import com.metamx.common.guava.CloseQuietly; -import io.druid.segment.CompressedPools; -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.FloatBuffer; -import java.nio.channels.Channels; -import java.util.Arrays; -import java.util.Collections; -import java.util.Random; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - -@RunWith(Parameterized.class) -public class CompressedFloatsIndexedSupplierTest extends CompressionStrategyTest -{ - public CompressedFloatsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) - { - super(compressionStrategy); - } - - private IndexedFloats indexed; - private CompressedFloatsIndexedSupplier supplier; - private float[] vals; - - @Before - public void setUp() throws Exception - { - Closeables.close(indexed, false); - indexed = null; - supplier = null; - vals = null; - } - - @After - public void tearDown() throws Exception - { - Closeables.close(indexed, false); - } - - private void setupSimple(final int chunkSize) - { - CloseQuietly.close(indexed); - - vals = new float[]{ - 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f, 0.16f - }; - - supplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( - FloatBuffer.wrap(vals), - chunkSize, - ByteOrder.nativeOrder(), - compressionStrategy - ); - - indexed = supplier.get(); - } - - private void setupSimpleWithSerde(final int chunkSize) throws IOException - { - vals = new float[]{ - 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f, 0.16f - }; - - makeWithSerde(chunkSize); - } - - private void makeWithSerde(int chunkSize) throws IOException - { - CloseQuietly.close(indexed); - - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final CompressedFloatsIndexedSupplier theSupplier = CompressedFloatsIndexedSupplier.fromFloatBuffer( - FloatBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy - ); - theSupplier.writeToChannel(Channels.newChannel(baos)); - - final byte[] bytes = baos.toByteArray(); - Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length); - - supplier = CompressedFloatsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), ByteOrder.nativeOrder()); - indexed = supplier.get(); - } - - private void setupLargeChunks(final int chunkSize, final int totalSize) throws IOException - { - vals = new float[totalSize]; - Random rand = new Random(0); - for(int i = 0; i < vals.length; ++i) { - vals[i] = (float)rand.nextGaussian(); - } - - makeWithSerde(chunkSize); - } - - @Test - public void testSanity() throws Exception - { - setupSimple(5); - Assert.assertEquals(4, supplier.getBaseFloatBuffers().size()); - assertIndexMatchesVals(); - - // test powers of 2 - setupSimple(2); - Assert.assertEquals(9, supplier.getBaseFloatBuffers().size()); - assertIndexMatchesVals(); - } - - @Test - public void testLargeChunks() throws Exception - { - final int maxChunkSize = CompressedPools.BUFFER_SIZE / Floats.BYTES; - - setupLargeChunks(maxChunkSize, 10 * maxChunkSize); - Assert.assertEquals(10, supplier.getBaseFloatBuffers().size()); - assertIndexMatchesVals(); - - setupLargeChunks(maxChunkSize, 10 * maxChunkSize + 1); - Assert.assertEquals(11, supplier.getBaseFloatBuffers().size()); - assertIndexMatchesVals(); - - setupLargeChunks(maxChunkSize - 1, 10 * (maxChunkSize - 1) + 1); - Assert.assertEquals(11, supplier.getBaseFloatBuffers().size()); - assertIndexMatchesVals(); - } - - @Test(expected = IllegalArgumentException.class) - public void testChunkTooBig() throws Exception - { - final int maxChunkSize = CompressedPools.BUFFER_SIZE / Floats.BYTES; - setupLargeChunks(maxChunkSize + 1, 10 * (maxChunkSize + 1)); - } - - @Test - public void testBulkFill() throws Exception - { - setupSimple(5); - - tryFill(0, 15); - tryFill(3, 6); - tryFill(7, 7); - tryFill(7, 9); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testBulkFillTooMuch() throws Exception - { - setupSimple(5); - tryFill(7, 11); - } - - @Test - public void testSanityWithSerde() throws Exception - { - setupSimpleWithSerde(5); - - Assert.assertEquals(4, supplier.getBaseFloatBuffers().size()); - - assertIndexMatchesVals(); - - // test powers of 2 - setupSimpleWithSerde(2); - - Assert.assertEquals(9, supplier.getBaseFloatBuffers().size()); - - assertIndexMatchesVals(); - } - - @Test - public void testBulkFillWithSerde() throws Exception - { - setupSimpleWithSerde(5); - - tryFill(0, 15); - tryFill(3, 6); - tryFill(7, 7); - tryFill(7, 9); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testBulkFillTooMuchWithSerde() throws Exception - { - setupSimpleWithSerde(5); - tryFill(7, 11); - } - - // This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it, - // which sucks but I can't think of a way to deterministically cause it... - @Test - public void testConcurrentThreadReads() throws Exception - { - setupSimple(5); - - 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(new Runnable() - { - @Override - public void run() - { - 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; j < indexed.size(); ++j) { - final float val = vals[j]; - final float indexedVal = indexed.get(j); - if (Floats.compare(val, indexedVal) != 0) { - failureHappened.set(true); - reason.set(String.format("Thread1[%d]: %f != %f", j, val, indexedVal)); - stopLatch.countDown(); - return; - } - } - } - } - catch (Exception e) { - e.printStackTrace(); - failureHappened.set(true); - reason.set(e.getMessage()); - } - - stopLatch.countDown(); - } - }).start(); - - final IndexedFloats indexed2 = supplier.get(); - try { - new Thread(new Runnable() - { - @Override - public void run() - { - try { - startLatch.await(); - } - catch (InterruptedException e) { - stopLatch.countDown(); - return; - } - - try { - for (int i = 0; i < numRuns; ++i) { - for (int j = indexed2.size() - 1; j >= 0; --j) { - final float val = vals[j]; - final float indexedVal = indexed2.get(j); - if (Floats.compare(val, indexedVal) != 0) { - failureHappened.set(true); - reason.set(String.format("Thread2[%d]: %f != %f", 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 { - indexed2.close(); - } - - if (failureHappened.get()) { - Assert.fail("Failure happened. Reason: " + reason.get()); - } - } - - private void tryFill(final int startIndex, final int size) - { - float[] filled = new float[size]; - indexed.fill(startIndex, filled); - - for (int i = startIndex; i < filled.length; i++) { - Assert.assertEquals(vals[i + startIndex], filled[i], 0.0); - } - } - - private void assertIndexMatchesVals() - { - Assert.assertEquals(vals.length, indexed.size()); - - // sequential access - int[] indices = new int[vals.length]; - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i), 0.0); - indices[i] = i; - } - - Collections.shuffle(Arrays.asList(indices)); - // random access - for (int i = 0; i < indexed.size(); ++i) { - int k = indices[i]; - Assert.assertEquals(vals[k], indexed.get(k), 0.0); - } - } -} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java new file mode 100644 index 000000000000..74f6d55bcd23 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java @@ -0,0 +1,295 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Supplier; +import com.google.common.io.ByteSink; +import com.google.common.primitives.Floats; +import com.metamx.common.guava.CloseQuietly; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +@RunWith(Parameterized.class) +public class CompressedFloatsSerdeTest +{ + @Parameterized.Parameters(name = "{0} {1} {2}") + public static Iterable compressionStrategies() + { + List data = new ArrayList<>(); + for (CompressedObjectStrategy.CompressionStrategy strategy : CompressedObjectStrategy.CompressionStrategy.values()) { + data.add(new Object[]{strategy, ByteOrder.BIG_ENDIAN}); + data.add(new Object[]{strategy, ByteOrder.LITTLE_ENDIAN}); + } + return data; + } + + private static final double DELTA = 0.00001; + + protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + protected final ByteOrder order; + + private final float values0[] = {}; + private final float values1[] = {0f, 1f, 1f, 0f, 1f, 1f, 1f, 1f, 0f, 0f, 1f, 1f}; + private final float values2[] = {13.2f, 6.1f, 0.001f, 123f, 12572f, 123.1f, 784.4f, 6892.8634f, 8.341111f}; + private final float values3[] = {0.001f, 0.001f, 0.001f, 0.001f, 0.001f, 100f, 100f, 100f, 100f, 100f}; + private final float values4[] = {0f, 0f, 0f, 0f, 0.01f, 0f, 0f, 0f, 21.22f, 0f, 0f, 0f, 0f, 0f, 0f}; + private final float values5[] = {123.16f, 1.12f, 62.00f, 462.12f, 517.71f, 56.54f, 971.32f, 824.22f, 472.12f, 625.26f}; + private final float values6[] = {1000000f, 1000001f, 1000002f, 1000003f, 1000004f, 1000005f, 1000006f, 1000007f, 1000008f}; + private final float values7[] = { + Float.MAX_VALUE, Float.MIN_VALUE, 12378.5734f, -12718243.7496f, -93653653.1f, 12743153.385534f, 21431.414538f, + 65487435436632.123f, -43734526234564.65f + }; + + public CompressedFloatsSerdeTest( + CompressedObjectStrategy.CompressionStrategy compressionStrategy, + ByteOrder order + ) + { + this.compressionStrategy = compressionStrategy; + this.order = order; + } + + @Test + public void testValueSerde() throws Exception + { + testWithValues(values0); + testWithValues(values1); + testWithValues(values2); + testWithValues(values3); + testWithValues(values4); + testWithValues(values5); + testWithValues(values6); + testWithValues(values7); + } + + @Test + public void testChunkSerde() throws Exception + { + float chunk[] = new float[10000]; + for (int i = 0; i < 10000; i++) { + chunk[i] = i; + } + testWithValues(chunk); + } + + public void testWithValues(float[] values) throws Exception + { + FloatSupplierSerializer serializer = CompressionFactory.getFloatSerializer(new IOPeonForTesting(), "test", order, compressionStrategy + ); + serializer.open(); + + for (float value : values) { + serializer.add(value); + } + Assert.assertEquals(values.length, serializer.size()); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.closeAndConsolidate( + new ByteSink() + { + @Override + public OutputStream openStream() throws IOException + { + return baos; + } + } + ); + Assert.assertEquals(baos.size(), serializer.getSerializedSize()); + CompressedFloatsIndexedSupplier supplier = CompressedFloatsIndexedSupplier + .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order); + IndexedFloats floats = supplier.get(); + + assertIndexMatchesVals(floats, values); + for (int i = 0; i < 10; i++) { + int a = (int) (Math.random() * values.length); + int b = (int) (Math.random() * values.length); + int start = a < b ? a : b; + int end = a < b ? b : a; + tryFill(floats, values, start, end - start); + } + testSupplierSerde(supplier, values); + testConcurrentThreadReads(supplier, floats, values); + + floats.close(); + } + + private void tryFill(IndexedFloats indexed, float[] vals, final int startIndex, final int size) + { + float[] filled = new float[size]; + indexed.fill(startIndex, filled); + + for (int i = startIndex; i < filled.length; i++) { + Assert.assertEquals(vals[i + startIndex], filled[i], DELTA); + } + } + + private void assertIndexMatchesVals(IndexedFloats indexed, float[] vals) + { + Assert.assertEquals(vals.length, indexed.size()); + + // sequential access + int[] indices = new int[vals.length]; + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i), DELTA); + indices[i] = i; + } + + Collections.shuffle(Arrays.asList(indices)); + // random access + for (int i = 0; i < indexed.size(); ++i) { + int k = indices[i]; + Assert.assertEquals(vals[k], indexed.get(k), DELTA); + } + } + + private void testSupplierSerde(CompressedFloatsIndexedSupplier supplier, float[] vals) throws IOException + { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + supplier.writeToChannel(Channels.newChannel(baos)); + + final byte[] bytes = baos.toByteArray(); + Assert.assertEquals(supplier.getSerializedSize(), bytes.length); + CompressedFloatsIndexedSupplier anotherSupplier = CompressedFloatsIndexedSupplier.fromByteBuffer( + ByteBuffer.wrap(bytes), order + ); + IndexedFloats indexed = anotherSupplier.get(); + assertIndexMatchesVals(indexed, vals); + } + + // This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it, + // which sucks but I can't think of a way to deterministically cause it... + private void testConcurrentThreadReads( + final Supplier supplier, + final IndexedFloats indexed, final float[] vals + ) throws Exception + { + 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(new Runnable() + { + @Override + public void run() + { + 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; j < indexed.size(); ++j) { + final float val = vals[j]; + final float indexedVal = indexed.get(j); + if (Floats.compare(val, indexedVal) != 0) { + failureHappened.set(true); + reason.set(String.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 IndexedFloats indexed2 = supplier.get(); + try { + new Thread(new Runnable() + { + @Override + public void run() + { + try { + startLatch.await(); + } + catch (InterruptedException e) { + stopLatch.countDown(); + return; + } + + try { + for (int i = 0; i < numRuns; ++i) { + for (int j = indexed2.size() - 1; j >= 0; --j) { + final float val = vals[j]; + final float indexedVal = indexed2.get(j); + if (Floats.compare(val, indexedVal) != 0) { + failureHappened.set(true); + reason.set(String.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(indexed2); + } + + if (failureHappened.get()) { + Assert.fail("Failure happened. Reason: " + reason.get()); + } + } +} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java deleted file mode 100644 index c9044dade899..000000000000 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSupplierSerializerTest.java +++ /dev/null @@ -1,136 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import com.google.common.io.OutputSupplier; -import io.druid.collections.ResourceHolder; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.FloatBuffer; - -@RunWith(Parameterized.class) -public class CompressedFloatsSupplierSerializerTest extends CompressionStrategyTest -{ - public CompressedFloatsSupplierSerializerTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) - { - super(compressionStrategy); - } - - @Test - public void testSanity() throws Exception - { - final ByteOrder order = ByteOrder.nativeOrder(); - final int sizePer = 999; - CompressedFloatsSupplierSerializer serializer = new CompressedFloatsSupplierSerializer( - sizePer, - new GenericIndexedWriter>( - new IOPeonForTesting(), - "test", - CompressedFloatBufferObjectStrategy.getBufferForOrder( - order, - compressionStrategy, - sizePer - ) - ), - compressionStrategy - ); - serializer.open(); - - final int numElements = 10000; - - for (int i = 0; i < numElements; ++i) { - serializer.add((float) i); - } - - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.closeAndConsolidate( - new OutputSupplier() - { - @Override - public OutputStream getOutput() throws IOException - { - return baos; - } - } - ); - - Assert.assertEquals(baos.size(), serializer.getSerializedSize()); - - IndexedFloats floats = CompressedFloatsIndexedSupplier - .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) - .get(); - - Assert.assertEquals(numElements, floats.size()); - for (int i = 0; i < numElements; ++i) { - Assert.assertEquals((float) i, floats.get(i), 0.0f); - } - - floats.close(); - } - - @Test - public void testEmpty() throws Exception - { - final ByteOrder order = ByteOrder.nativeOrder(); - final int sizePer = 999; - CompressedFloatsSupplierSerializer serializer = new CompressedFloatsSupplierSerializer( - sizePer, - new GenericIndexedWriter>( - new IOPeonForTesting(), - "test", - CompressedFloatBufferObjectStrategy.getBufferForOrder( - order, - compressionStrategy, - sizePer - ) - ), - compressionStrategy - ); - serializer.open(); - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.closeAndConsolidate( - new OutputSupplier() - { - @Override - public OutputStream getOutput() throws IOException - { - return baos; - } - } - ); - - Assert.assertEquals(baos.size(), serializer.getSerializedSize()); - IndexedFloats floats = CompressedFloatsIndexedSupplier - .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) - .get(); - - Assert.assertEquals(0, floats.size()); - floats.close(); - } - -} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java index 8b695df4487a..f53323f1364e 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java @@ -49,7 +49,7 @@ public class CompressedIntsIndexedWriterTest public static Iterable compressionStrategiesAndByteOrders() { Set> combinations = Sets.cartesianProduct( - Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()), + Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) ); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java deleted file mode 100644 index d8fd4946d3f5..000000000000 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsIndexedSupplierTest.java +++ /dev/null @@ -1,343 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import com.google.common.primitives.Longs; -import com.metamx.common.guava.CloseQuietly; -import io.druid.segment.CompressedPools; -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.LongBuffer; -import java.nio.channels.Channels; -import java.util.Arrays; -import java.util.Collections; -import java.util.Random; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - -@RunWith(Parameterized.class) -public class CompressedLongsIndexedSupplierTest extends CompressionStrategyTest -{ - public CompressedLongsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) - { - super(compressionStrategy); - } - - private IndexedLongs indexed; - private CompressedLongsIndexedSupplier supplier; - private long[] vals; - - @Before - public void setUp() throws Exception - { - CloseQuietly.close(indexed); - indexed = null; - supplier = null; - vals = null; - } - - @After - public void tearDown() throws Exception - { - CloseQuietly.close(indexed); - } - - private void setupSimple(final int chunkSize) - { - CloseQuietly.close(indexed); - - vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16}; - - supplier = CompressedLongsIndexedSupplier.fromLongBuffer( - LongBuffer.wrap(vals), - chunkSize, - ByteOrder.nativeOrder(), - compressionStrategy - ); - - indexed = supplier.get(); - } - - private void setupSimpleWithSerde(final int chunkSize) throws IOException - { - vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16}; - - makeWithSerde(chunkSize); - } - - private void makeWithSerde(final int chunkSize) throws IOException - { - CloseQuietly.close(indexed); - - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - final CompressedLongsIndexedSupplier theSupplier = CompressedLongsIndexedSupplier.fromLongBuffer( - LongBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy - ); - theSupplier.writeToChannel(Channels.newChannel(baos)); - - final byte[] bytes = baos.toByteArray(); - Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length); - - supplier = CompressedLongsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), ByteOrder.nativeOrder()); - indexed = supplier.get(); - } - - private void setupLargeChunks(final int chunkSize, final int totalSize) throws IOException - { - vals = new long[totalSize]; - Random rand = new Random(0); - for(int i = 0; i < vals.length; ++i) { - vals[i] = rand.nextLong(); - } - - makeWithSerde(chunkSize); - } - - @Test - public void testSanity() throws Exception - { - setupSimple(5); - - Assert.assertEquals(4, supplier.getBaseLongBuffers().size()); - assertIndexMatchesVals(); - - // test powers of 2 - setupSimple(4); - Assert.assertEquals(4, supplier.getBaseLongBuffers().size()); - assertIndexMatchesVals(); - - setupSimple(32); - Assert.assertEquals(1, supplier.getBaseLongBuffers().size()); - assertIndexMatchesVals(); - } - - @Test - public void testLargeChunks() throws Exception - { - final int maxChunkSize = CompressedPools.BUFFER_SIZE / Longs.BYTES; - - setupLargeChunks(maxChunkSize, 10 * maxChunkSize); - Assert.assertEquals(10, supplier.getBaseLongBuffers().size()); - assertIndexMatchesVals(); - - setupLargeChunks(maxChunkSize, 10 * maxChunkSize + 1); - Assert.assertEquals(11, supplier.getBaseLongBuffers().size()); - assertIndexMatchesVals(); - - setupLargeChunks(maxChunkSize - 1, 10 * (maxChunkSize - 1) + 1); - Assert.assertEquals(11, supplier.getBaseLongBuffers().size()); - assertIndexMatchesVals(); - } - - @Test(expected = IllegalArgumentException.class) - public void testChunkTooBig() throws Exception - { - final int maxChunkSize = CompressedPools.BUFFER_SIZE / Longs.BYTES; - setupLargeChunks(maxChunkSize + 1, 10 * (maxChunkSize + 1)); - } - - @Test - public void testBulkFill() throws Exception - { - setupSimple(5); - - tryFill(0, 15); - tryFill(3, 6); - tryFill(7, 7); - tryFill(7, 9); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testBulkFillTooMuch() throws Exception - { - setupSimple(5); - tryFill(7, 10); - } - - @Test - public void testSanityWithSerde() throws Exception - { - setupSimpleWithSerde(5); - - Assert.assertEquals(4, supplier.getBaseLongBuffers().size()); - assertIndexMatchesVals(); - } - - @Test - public void testBulkFillWithSerde() throws Exception - { - setupSimpleWithSerde(5); - - tryFill(0, 15); - tryFill(3, 6); - tryFill(7, 7); - tryFill(7, 9); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testBulkFillTooMuchWithSerde() throws Exception - { - setupSimpleWithSerde(5); - tryFill(7, 10); - } - - // This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it, - // which sucks but I can't think of a way to deterministically cause it... - @Test - public void testConcurrentThreadReads() throws Exception - { - setupSimple(5); - - 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(new Runnable() - { - @Override - public void run() - { - 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; j < indexed.size(); ++j) { - final long val = vals[j]; - final long indexedVal = indexed.get(j); - if (Longs.compare(val, indexedVal) != 0) { - failureHappened.set(true); - reason.set(String.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 IndexedLongs indexed2 = supplier.get(); - try { - new Thread(new Runnable() - { - @Override - public void run() - { - try { - startLatch.await(); - } - catch (InterruptedException e) { - stopLatch.countDown(); - return; - } - - try { - for (int i = 0; i < numRuns; ++i) { - for (int j = indexed2.size() - 1; j >= 0; --j) { - final long val = vals[j]; - final long indexedVal = indexed2.get(j); - if (Longs.compare(val, indexedVal) != 0) { - failureHappened.set(true); - reason.set(String.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(indexed2); - } - - if (failureHappened.get()) { - Assert.fail("Failure happened. Reason: " + reason.get()); - } - } - - private void tryFill(final int startIndex, final int size) - { - long[] filled = new long[size]; - indexed.fill(startIndex, filled); - - for (int i = startIndex; i < filled.length; i++) { - Assert.assertEquals(vals[i + startIndex], filled[i]); - } - } - - private void assertIndexMatchesVals() - { - Assert.assertEquals(vals.length, indexed.size()); - - // sequential access - int[] indices = new int[vals.length]; - for (int i = 0; i < indexed.size(); ++i) { - Assert.assertEquals(vals[i], indexed.get(i), 0.0); - indices[i] = i; - } - - Collections.shuffle(Arrays.asList(indices)); - // random access - for (int i = 0; i < indexed.size(); ++i) { - int k = indices[i]; - Assert.assertEquals(vals[k], indexed.get(k), 0.0); - } - } -} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java new file mode 100644 index 000000000000..0e9235d65d56 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java @@ -0,0 +1,317 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + +import com.google.common.base.Supplier; +import com.google.common.io.ByteSink; +import com.google.common.primitives.Longs; +import com.metamx.common.guava.CloseQuietly; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +@RunWith(Parameterized.class) +public class CompressedLongsSerdeTest +{ + @Parameterized.Parameters(name = "{0} {1} {2}") + public static Iterable compressionStrategies() + { + List data = new ArrayList<>(); + for (CompressionFactory.LongEncodingStrategy encodingStrategy: CompressionFactory.LongEncodingStrategy.values()) { + for (CompressedObjectStrategy.CompressionStrategy strategy : CompressedObjectStrategy.CompressionStrategy.values()) { + data.add(new Object[]{encodingStrategy, strategy, ByteOrder.BIG_ENDIAN}); + data.add(new Object[]{encodingStrategy, strategy, ByteOrder.LITTLE_ENDIAN}); + } + } + return data; + } + + protected final CompressionFactory.LongEncodingStrategy encodingStrategy; + protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + protected final ByteOrder order; + + private final long values0[] = {}; + private final long values1[] = {0, 1, 1, 0, 1, 1, 1, 1, 0, 0, 1, 1}; + private final long values2[] = {12, 5, 2, 9, 3, 2, 5, 1, 0, 6, 13, 10, 15}; + private final long values3[] = {1, 1, 1, 1, 1, 11, 11, 11, 11}; + private final long values4[] = {200, 200, 200, 401, 200, 301, 200, 200, 200, 404, 200, 200, 200, 200}; + private final long values5[] = {123, 632, 12, 39, 536, 0, 1023, 52, 777, 526, 214, 562, 823, 346}; + private final long values6[] = {1000000, 1000001, 1000002, 1000003, 1000004, 1000005, 1000006, 1000007, 1000008}; + private final long values7[] = { + Long.MAX_VALUE, Long.MIN_VALUE, 12378, -12718243, -1236213, 12743153, 21364375452L, + 65487435436632L, -43734526234564L + }; + private final long values8[] = {Long.MAX_VALUE, 0, 321, 15248425, 13523212136L, 63822, 3426, 96}; + + // built test value with enough unique values to not use table encoding for auto strategy + private static long[] addUniques(long[] val) { + long[] ret = new long[val.length + CompressionFactory.MAX_TABLE_SIZE]; + for (int i = 0; i < CompressionFactory.MAX_TABLE_SIZE; i++) { + ret[i] = i; + } + System.arraycopy(val, 0, ret, 256, val.length); + return ret; + } + + public CompressedLongsSerdeTest( + CompressionFactory.LongEncodingStrategy encodingStrategy, + CompressedObjectStrategy.CompressionStrategy compressionStrategy, + ByteOrder order + ) + { + this.encodingStrategy = encodingStrategy; + this.compressionStrategy = compressionStrategy; + this.order = order; + } + + @Test + public void testValueSerde() throws Exception + { + testWithValues(values0); + testWithValues(values1); + testWithValues(values2); + testWithValues(values3); + testWithValues(values4); + testWithValues(values5); + testWithValues(values6); + testWithValues(values7); + testWithValues(values8); + } + + @Test + public void testChunkSerde() throws Exception + { + long chunk[] = new long[10000]; + for (int i = 0; i < 10000; i++) { + chunk[i] = i; + } + testWithValues(chunk); + } + + public void testWithValues(long[] values) throws Exception + { + testValues(values); + testValues(addUniques(values)); + } + + public void testValues(long[] values) throws Exception + { + LongSupplierSerializer serializer = CompressionFactory.getLongSerializer(new IOPeonForTesting(), "test", order, + encodingStrategy, compressionStrategy + ); + serializer.open(); + + for (long value : values) { + serializer.add(value); + } + Assert.assertEquals(values.length, serializer.size()); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.closeAndConsolidate( + new ByteSink() + { + @Override + public OutputStream openStream() throws IOException + { + return baos; + } + } + ); + Assert.assertEquals(baos.size(), serializer.getSerializedSize()); + CompressedLongsIndexedSupplier supplier = CompressedLongsIndexedSupplier + .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order); + IndexedLongs longs = supplier.get(); + + assertIndexMatchesVals(longs, values); + for (int i = 0; i < 10; i++) { + int a = (int) (Math.random() * values.length); + int b = (int) (Math.random() * values.length); + int start = a < b ? a : b; + int end = a < b ? b : a; + tryFill(longs, values, start, end - start); + } + testSupplierSerde(supplier, values); + testConcurrentThreadReads(supplier, longs, values); + + longs.close(); + } + + private void tryFill(IndexedLongs indexed, long[] vals, final int startIndex, final int size) + { + long[] filled = new long[size]; + indexed.fill(startIndex, filled); + + for (int i = startIndex; i < filled.length; i++) { + Assert.assertEquals(vals[i + startIndex], filled[i]); + } + } + + private void assertIndexMatchesVals(IndexedLongs indexed, long[] vals) + { + Assert.assertEquals(vals.length, indexed.size()); + + // sequential access + int[] indices = new int[vals.length]; + for (int i = 0; i < indexed.size(); ++i) { + Assert.assertEquals(vals[i], indexed.get(i)); + indices[i] = i; + } + + Collections.shuffle(Arrays.asList(indices)); + // random access + for (int i = 0; i < indexed.size(); ++i) { + int k = indices[i]; + Assert.assertEquals(vals[k], indexed.get(k)); + } + } + + private void testSupplierSerde(CompressedLongsIndexedSupplier supplier, long[] vals) throws IOException + { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + supplier.writeToChannel(Channels.newChannel(baos)); + + final byte[] bytes = baos.toByteArray(); + Assert.assertEquals(supplier.getSerializedSize(), bytes.length); + CompressedLongsIndexedSupplier anotherSupplier = CompressedLongsIndexedSupplier.fromByteBuffer( + ByteBuffer.wrap(bytes), order + ); + IndexedLongs indexed = anotherSupplier.get(); + assertIndexMatchesVals(indexed, vals); + } + + // This test attempts to cause a race condition with the DirectByteBuffers, it's non-deterministic in causing it, + // which sucks but I can't think of a way to deterministically cause it... + private void testConcurrentThreadReads( + final Supplier supplier, + final IndexedLongs indexed, final long[] vals + ) throws Exception + { + 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(new Runnable() + { + @Override + public void run() + { + 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; j < indexed.size(); ++j) { + final long val = vals[j]; + final long indexedVal = indexed.get(j); + if (Longs.compare(val, indexedVal) != 0) { + failureHappened.set(true); + reason.set(String.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 IndexedLongs indexed2 = supplier.get(); + try { + new Thread(new Runnable() + { + @Override + public void run() + { + try { + startLatch.await(); + } + catch (InterruptedException e) { + stopLatch.countDown(); + return; + } + + try { + for (int i = 0; i < numRuns; ++i) { + for (int j = indexed2.size() - 1; j >= 0; --j) { + final long val = vals[j]; + final long indexedVal = indexed2.get(j); + if (Longs.compare(val, indexedVal) != 0) { + failureHappened.set(true); + reason.set(String.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(indexed2); + } + + if (failureHappened.get()) { + Assert.fail("Failure happened. Reason: " + reason.get()); + } + } +} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java deleted file mode 100644 index eef046409482..000000000000 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSupplierSerializerTest.java +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import com.google.common.io.OutputSupplier; -import io.druid.collections.ResourceHolder; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.LongBuffer; - -@RunWith(Parameterized.class) -public class CompressedLongsSupplierSerializerTest extends CompressionStrategyTest -{ - public CompressedLongsSupplierSerializerTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) - { - super(compressionStrategy); - } - - @Test - public void testSanity() throws Exception - { - final ByteOrder order = ByteOrder.nativeOrder(); - final int sizePer = 999; - CompressedLongsSupplierSerializer serializer = new CompressedLongsSupplierSerializer( - sizePer, - new GenericIndexedWriter>( - new IOPeonForTesting(), - "test", - CompressedLongBufferObjectStrategy.getBufferForOrder(order, compressionStrategy, sizePer) - ), - compressionStrategy - ); - serializer.open(); - - final int numElements = 10000; - - for (int i = 0; i < numElements; ++i) { - serializer.add((long) i); - } - - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.closeAndConsolidate( - new OutputSupplier() - { - @Override - public OutputStream getOutput() throws IOException - { - return baos; - } - } - ); - - Assert.assertEquals(baos.size(), serializer.getSerializedSize()); - - IndexedLongs longs = CompressedLongsIndexedSupplier - .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) - .get(); - - Assert.assertEquals(numElements, longs.size()); - for (int i = 0; i < numElements; ++i) { - Assert.assertEquals((long) i, longs.get(i), 0.0f); - } - longs.close(); - } - - @Test - public void testEmpty() throws Exception - { - final ByteOrder order = ByteOrder.nativeOrder(); - final int sizePer = 999; - CompressedLongsSupplierSerializer serializer = new CompressedLongsSupplierSerializer( - sizePer, - new GenericIndexedWriter>( - new IOPeonForTesting(), - "test", - CompressedLongBufferObjectStrategy.getBufferForOrder(order, compressionStrategy, sizePer) - ), - compressionStrategy - ); - serializer.open(); - final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.closeAndConsolidate( - new OutputSupplier() - { - @Override - public OutputStream getOutput() throws IOException - { - return baos; - } - } - ); - Assert.assertEquals(baos.size(), serializer.getSerializedSize()); - - IndexedLongs longs = CompressedLongsIndexedSupplier - .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order) - .get(); - - Assert.assertEquals(0, longs.size()); - longs.close(); - } -} diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java index 4f44b11ff25f..77ec47881b01 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -53,7 +53,7 @@ public class CompressedVSizeIndexedV3WriterTest public static Iterable compressionStrategiesAndByteOrders() { Set> combinations = Sets.cartesianProduct( - Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()), + Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) ); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java index f6fd2ec9e429..720cd64a5117 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java @@ -49,7 +49,7 @@ public class CompressedVSizeIntsIndexedWriterTest public static Iterable compressionStrategiesAndByteOrders() { Set> combinations = Sets.cartesianProduct( - Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.values()), + Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) ); diff --git a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java index 7a89d63c65d2..4867cd27689b 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java @@ -52,7 +52,7 @@ public class CompressionStrategyTest public static Iterable compressionStrategies() { return Iterables.transform( - Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values()), + Arrays.asList(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), new Function() { @Override diff --git a/processing/src/test/java/io/druid/segment/data/InMemoryCompressedFloatsTest.java b/processing/src/test/java/io/druid/segment/data/InMemoryCompressedFloatsTest.java deleted file mode 100644 index b36447902671..000000000000 --- a/processing/src/test/java/io/druid/segment/data/InMemoryCompressedFloatsTest.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.nio.ByteOrder; - -/** - */ -public class InMemoryCompressedFloatsTest -{ - private InMemoryCompressedFloats floats; - private float[] vals; - - @Before - public void setUp() throws Exception - { - floats = null; - vals = null; - } - - private void setupSimple() - { - vals = new float[]{ - 0.0f, 0.1f, 0.2f, 0.3f, 0.4f, 0.5f, 0.6f, 0.7f, 0.8f, 0.9f, 0.10f, 0.11f, 0.12f, 0.13f, 0.14f, 0.15f - }; - - floats = new InMemoryCompressedFloats(5, ByteOrder.nativeOrder()); - - for (int i = 0; i < vals.length; i++) { - Assert.assertEquals(i, floats.add(vals[i])); - } - } - - @Test - public void testSanity() throws Exception - { - setupSimple(); - - Assert.assertEquals(vals.length, this.floats.size()); - for (int i = 0; i < this.floats.size(); ++i) { - Assert.assertEquals(vals[i], this.floats.get(i), 0.0); - } - } - - @Test - public void testBulkFill() throws Exception - { - setupSimple(); - - tryFill(0, 16); - tryFill(3, 6); - tryFill(7, 7); - tryFill(7, 9); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testBulkFillTooMuch() throws Exception - { - setupSimple(); - tryFill(7, 10); - } - - private void tryFill(final int startIndex, final int size) - { - float[] filled = new float[size]; - this.floats.fill(startIndex, filled); - - for (int i = startIndex; i < filled.length; i++) { - Assert.assertEquals(vals[i + startIndex], filled[i], 0.0); - } - } - - @Test - public void testCanConvertToCompressedFloatsIndexedSupplier() throws Exception - { - setupSimple(); - - IndexedFloats indexed = floats.toCompressedFloatsIndexedSupplier().get(); - - for (int i = 0; i < floats.size(); i++) { - Assert.assertEquals(floats.get(i), indexed.get(i), 0.0000f); - } - - indexed.close(); - } -} diff --git a/processing/src/test/java/io/druid/segment/data/InMemoryCompressedLongsTest.java b/processing/src/test/java/io/druid/segment/data/InMemoryCompressedLongsTest.java deleted file mode 100644 index e6be13cff17b..000000000000 --- a/processing/src/test/java/io/druid/segment/data/InMemoryCompressedLongsTest.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; - -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.nio.ByteOrder; - -/** - */ -public class InMemoryCompressedLongsTest -{ - private InMemoryCompressedLongs longs; - private long[] vals; - - @Before - public void setUp() throws Exception - { - longs = null; - vals = null; - } - - private void setupSimple() - { - vals = new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16}; - - longs = new InMemoryCompressedLongs( - 5, - ByteOrder.nativeOrder() - ); - - for (int i = 0; i < vals.length; i++) { - Assert.assertEquals(i, longs.add(vals[i])); - } - } - - @Test - public void testSanity() throws Exception - { - setupSimple(); - - Assert.assertEquals(vals.length, longs.size()); - for (int i = 0; i < longs.size(); ++i) { - Assert.assertEquals(vals[i], longs.get(i)); - } - } - - @Test - public void testBulkFill() throws Exception - { - setupSimple(); - - tryFill(0, 16); - tryFill(3, 6); - tryFill(7, 7); - tryFill(7, 9); - } - - @Test(expected = IndexOutOfBoundsException.class) - public void testBulkFillTooMuch() throws Exception - { - setupSimple(); - tryFill(7, 10); - } - - private void tryFill(final int startIndex, final int size) - { - long[] filled = new long[size]; - longs.fill(startIndex, filled); - - for (int i = startIndex; i < filled.length; i++) { - Assert.assertEquals(vals[i + startIndex], filled[i]); - } - } - - @Test - public void testCanConvertToCompressedLongsIndexedSupplier() throws Exception - { - setupSimple(); - - IndexedLongs indexed = longs.toCompressedLongsIndexedSupplier().get(); - - for (int i = 0; i < longs.size(); i++) { - Assert.assertEquals(longs.get(i), indexed.get(i)); - } - - indexed.close(); - } -} diff --git a/processing/src/test/java/io/druid/segment/data/VSizeLongSerdeTest.java b/processing/src/test/java/io/druid/segment/data/VSizeLongSerdeTest.java new file mode 100644 index 000000000000..fea87e9956ec --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/VSizeLongSerdeTest.java @@ -0,0 +1,151 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.segment.data; + + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +public class VSizeLongSerdeTest +{ + private ByteBuffer buffer; + private ByteArrayOutputStream outStream; + private ByteBuffer outBuffer; + private long values0[] = {0, 1, 1, 0, 1, 1, 1, 1, 0, 0, 1, 1}; + private long values1[] = {0, 1, 1, 0, 1, 1, 1, 1, 0, 0, 1, 1}; + private long values2[] = {12, 5, 2, 9, 3, 2, 5, 1, 0, 6, 13, 10, 15}; + private long values3[] = {1, 1, 1, 1, 1, 11, 11, 11, 11}; + private long values4[] = {200, 200, 200, 401, 200, 301, 200, 200, 200, 404, 200, 200, 200, 200}; + private long values5[] = {123, 632, 12, 39, 536, 0, 1023, 52, 777, 526, 214, 562, 823, 346}; + private long values6[] = {1000000, 1000001, 1000002, 1000003, 1000004, 1000005, 1000006, 1000007, 1000008}; + + @Before + public void setUp() + { + outStream = new ByteArrayOutputStream(); + outBuffer = ByteBuffer.allocate(500000); + } + + @Test + public void testGetBitsForMax() + { + Assert.assertEquals(1, VSizeLongSerde.getBitsForMax(1)); + Assert.assertEquals(1, VSizeLongSerde.getBitsForMax(2)); + Assert.assertEquals(2, VSizeLongSerde.getBitsForMax(3)); + Assert.assertEquals(4, VSizeLongSerde.getBitsForMax(16)); + Assert.assertEquals(8, VSizeLongSerde.getBitsForMax(200)); + Assert.assertEquals(12, VSizeLongSerde.getBitsForMax(999)); + Assert.assertEquals(24, VSizeLongSerde.getBitsForMax(12345678)); + Assert.assertEquals(32, VSizeLongSerde.getBitsForMax(Integer.MAX_VALUE)); + Assert.assertEquals(64, VSizeLongSerde.getBitsForMax(Long.MAX_VALUE)); + } + + @Test + public void testSerdeValues() throws IOException + { + for (int i : VSizeLongSerde.SUPPORTED_SIZE) { + testSerde(i, values0); + if (i >= 1) { + testSerde(i, values1); + } + if (i >= 4) { + testSerde(i, values2); + testSerde(i, values3); + } + if (i >= 9) { + testSerde(i, values4); + } + if (i >= 10) { + testSerde(i, values5); + } + if (i >= 20) { + testSerde(i, values6); + } + } + } + + @Test + public void testSerdeLoop() throws IOException + { + for (int i : VSizeLongSerde.SUPPORTED_SIZE) { + if (i >= 8) { + testSerdeIncLoop(i, 0, 256); + } + if (i >= 16) { + testSerdeIncLoop(i, 0, 50000); + } + } + } + + public void testSerde(int longSize, long[] values) throws IOException + { + outBuffer.rewind(); + outStream.reset(); + VSizeLongSerde.LongSerializer streamSer = VSizeLongSerde.getSerializer(longSize, outStream); + VSizeLongSerde.LongSerializer bufferSer = VSizeLongSerde.getSerializer(longSize, outBuffer, 0); + for (long value : values) { + streamSer.write(value); + bufferSer.write(value); + } + streamSer.close(); + bufferSer.close(); + + buffer = ByteBuffer.wrap(outStream.toByteArray()); + Assert.assertEquals(VSizeLongSerde.getSerializedSize(longSize, values.length), buffer.capacity()); + Assert.assertEquals(VSizeLongSerde.getSerializedSize(longSize, values.length), outBuffer.position()); + VSizeLongSerde.LongDeserializer streamDes = VSizeLongSerde.getDeserializer(longSize, buffer, 0); + VSizeLongSerde.LongDeserializer bufferDes = VSizeLongSerde.getDeserializer(longSize, outBuffer, 0); + for (int i = 0; i < values.length; i++) { + Assert.assertEquals(values[i], streamDes.get(i)); + Assert.assertEquals(values[i], bufferDes.get(i)); + } + } + + public void testSerdeIncLoop(int longSize, long start, long end) throws IOException + { + outBuffer.rewind(); + outStream.reset(); + VSizeLongSerde.LongSerializer streamSer = VSizeLongSerde.getSerializer(longSize, outStream); + VSizeLongSerde.LongSerializer bufferSer = VSizeLongSerde.getSerializer(longSize, outBuffer, 0); + for (long i = start; i < end; i++) { + streamSer.write(i); + bufferSer.write(i); + } + streamSer.close(); + bufferSer.close(); + + buffer = ByteBuffer.wrap(outStream.toByteArray()); + Assert.assertEquals(VSizeLongSerde.getSerializedSize(longSize, (int) (end - start)), buffer.capacity()); + Assert.assertEquals(VSizeLongSerde.getSerializedSize(longSize, (int) (end - start)), outBuffer.position()); + VSizeLongSerde.LongDeserializer streamDes = VSizeLongSerde.getDeserializer(longSize, buffer, 0); + VSizeLongSerde.LongDeserializer bufferDes = VSizeLongSerde.getDeserializer(longSize, outBuffer, 0); + for (int i = 0; i < end - start; i++) { + Assert.assertEquals(start + i, streamDes.get(i)); + Assert.assertEquals(start + i, bufferDes.get(i)); + } + } + + +} diff --git a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java index 3e048c50223e..6b40ad840435 100644 --- a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java @@ -239,6 +239,7 @@ public void close() throws IOException .indexSpec(new IndexSpec( bitmapSerdeFactoryEntry.getValue(), null, + null, null )) .indexMerger(indexMergerEntry.getValue()); diff --git a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java index 710d852aeef4..6ade6eb34152 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java @@ -23,6 +23,7 @@ import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.data.IndexedInts; @@ -37,8 +38,9 @@ public class IncrementalIndexAdapterTest { private static final IndexSpec INDEX_SPEC = new IndexSpec( new ConciseBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4.name().toLowerCase(), - CompressedObjectStrategy.CompressionStrategy.LZ4.name().toLowerCase() + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionFactory.LongEncodingStrategy.LONGS ); @Test