diff --git a/benchmarks/src/main/java/io/druid/benchmark/CompressedIndexedIntsBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/CompressedIndexedIntsBenchmark.java index 307ef5d67d4..23c4adb57dd 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/CompressedIndexedIntsBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/CompressedIndexedIntsBenchmark.java @@ -19,12 +19,13 @@ package io.druid.benchmark; -import com.google.common.primitives.Ints; -import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.java.util.common.io.Closer; import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.VSizeIndexedInts; import io.druid.segment.data.WritableSupplier; +import it.unimi.dsi.fastutil.ints.IntArrayList; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Mode; @@ -69,25 +70,20 @@ public class CompressedIndexedIntsBenchmark } final ByteBuffer bufferCompressed = serialize( CompressedVSizeIntsIndexedSupplier.fromList( - Ints.asList(vals), + IntArrayList.wrap(vals), bound - 1, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForBytes(bytes), - ByteOrder.nativeOrder(), CompressedObjectStrategy.CompressionStrategy.LZ4 + ByteOrder.nativeOrder(), + CompressionStrategy.LZ4, + Closer.create() ) ); this.compressed = CompressedVSizeIntsIndexedSupplier.fromByteBuffer( bufferCompressed, - ByteOrder.nativeOrder(), - null + ByteOrder.nativeOrder() ).get(); - final ByteBuffer bufferUncompressed = serialize( - new VSizeIndexedInts.VSizeIndexedIntsSupplier( - VSizeIndexedInts.fromArray( - vals - ) - ) - ); + final ByteBuffer bufferUncompressed = serialize(VSizeIndexedInts.fromArray(vals)); this.uncompressed = VSizeIndexedInts.readFromByteBuffer(bufferUncompressed); filter = new BitSet(); @@ -128,7 +124,7 @@ public class CompressedIndexedIntsBenchmark } }; - writableSupplier.writeToChannel(channel); + writableSupplier.writeTo(channel, null); buffer.rewind(); return buffer; } diff --git a/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java index c0b1c7f1097..f829bccf939 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java @@ -22,8 +22,9 @@ package io.druid.benchmark; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import io.druid.java.util.common.io.Closer; import io.druid.segment.CompressedVSizeIndexedSupplier; -import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedMultivalue; import io.druid.segment.data.VSizeIndexed; @@ -95,13 +96,14 @@ public class CompressedVSizeIndexedBenchmark } ), bound - 1, - ByteOrder.nativeOrder(), CompressedObjectStrategy.CompressionStrategy.LZ4 + ByteOrder.nativeOrder(), + CompressionStrategy.LZ4, + Closer.create() ) ); this.compressed = CompressedVSizeIndexedSupplier.fromByteBuffer( bufferCompressed, - ByteOrder.nativeOrder(), - null + ByteOrder.nativeOrder() ).get(); final ByteBuffer bufferUncompressed = serialize( @@ -117,7 +119,7 @@ public class CompressedVSizeIndexedBenchmark } } ) - ).asWritableSupplier() + ) ); this.uncompressed = VSizeIndexed.readFromByteBuffer(bufferUncompressed); @@ -159,7 +161,7 @@ public class CompressedVSizeIndexedBenchmark } }; - writableSupplier.writeToChannel(channel); + writableSupplier.writeTo(channel, null); buffer.rewind(); return buffer; } diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java index ca011055bb9..b409b9cb6d5 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java @@ -36,6 +36,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.js.JavaScriptConfig; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.extraction.ExtractionFn; @@ -132,6 +133,7 @@ public class FilterPartitionBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -141,7 +143,7 @@ public class FilterPartitionBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup @@ -178,7 +180,8 @@ public class FilterPartitionBenchmark indexFile = INDEX_MERGER_V9.persist( incIndex, tmpDir, - new IndexSpec() + new IndexSpec(), + null ); qIndex = INDEX_IO.loadIndex(indexFile); diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java index 5763fa9b12b..4b9422ae6cc 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java @@ -35,6 +35,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.js.JavaScriptConfig; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -128,6 +129,7 @@ public class FilteredAggregatorBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -137,7 +139,7 @@ public class FilteredAggregatorBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup @@ -194,7 +196,8 @@ public class FilteredAggregatorBenchmark indexFile = INDEX_MERGER_V9.persist( incIndex, tmpDir, - new IndexSpec() + new IndexSpec(), + null ); qIndex = INDEX_IO.loadIndex(indexFile); diff --git a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java index b06ecb0657a..ee46f8384f7 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmark.java @@ -72,7 +72,7 @@ public class FloatCompressionBenchmark File compFile = new File(dir, file + "-" + strategy); rand = new Random(); ByteBuffer buffer = Files.map(compFile); - supplier = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder(), null); + supplier = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder()); } @Benchmark diff --git a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java index f66a14f420b..0f1eaf76d34 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java @@ -20,24 +20,20 @@ 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.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.FloatSupplierSerializer; -import io.druid.segment.data.TmpFileIOPeon; import java.io.BufferedReader; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; -import java.io.OutputStream; import java.io.Writer; import java.net.URISyntaxException; -import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.FileChannel; import java.nio.charset.StandardCharsets; @@ -51,10 +47,10 @@ public class FloatCompressionBenchmarkFileGenerator { private static final Logger log = new Logger(FloatCompressionBenchmarkFileGenerator.class); public static final int ROW_NUM = 5000000; - public static final List compressions = + public static final List compressions = ImmutableList.of( - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.NONE + CompressionStrategy.LZ4, + CompressionStrategy.NONE ); private static String dirPath = "floatCompress/"; @@ -143,48 +139,30 @@ public class FloatCompressionBenchmarkFileGenerator // create compressed files using all combinations of CompressionStrategy and FloatEncoding provided for (Map.Entry entry : generators.entrySet()) { - for (CompressedObjectStrategy.CompressionStrategy compression : compressions) { + for (CompressionStrategy compression : compressions) { String name = entry.getKey() + "-" + compression.toString(); log.info("%s: ", 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, + new OffHeapMemorySegmentWriteOutMedium(), "float", ByteOrder.nativeOrder(), compression ); - BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8); - - try (FileChannel output = FileChannel.open( - compFile.toPath(), - StandardOpenOption.CREATE_NEW, - StandardOpenOption.WRITE - )) { + try ( + BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8); + 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.close(); - br.close(); + writer.writeTo(output, null); } log.info("%d", compFile.length() / 1024); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java index 4b5fb28574d..2d9a60c559e 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java @@ -23,10 +23,10 @@ import com.google.common.io.Files; import com.google.common.primitives.Ints; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.ObjectStrategy; -import io.druid.segment.data.TmpFileIOPeon; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -106,7 +106,7 @@ public class GenericIndexedBenchmark public void createGenericIndexed() throws IOException { GenericIndexedWriter genericIndexedWriter = new GenericIndexedWriter<>( - new TmpFileIOPeon(), + new OffHeapMemorySegmentWriteOutMedium(), "genericIndexedBenchmark", byteArrayStrategy ); @@ -121,14 +121,13 @@ public class GenericIndexedBenchmark element.putInt(0, i); genericIndexedWriter.write(element.array()); } - genericIndexedWriter.close(); smooshDir = Files.createTempDir(); file = File.createTempFile("genericIndexedBenchmark", "meta"); try (FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE); FileSmoosher fileSmoosher = new FileSmoosher(smooshDir)) { - genericIndexedWriter.writeToChannel(fileChannel, fileSmoosher); + genericIndexedWriter.writeTo(fileChannel, fileSmoosher); } FileChannel fileChannel = FileChannel.open(file.toPath()); diff --git a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java index 4cdacaf3bc5..e8160672d7c 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -46,6 +46,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -153,6 +154,7 @@ public class GroupByTypeInterfaceBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -162,7 +164,7 @@ public class GroupByTypeInterfaceBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -339,7 +341,8 @@ public class GroupByTypeInterfaceBenchmark final File file = INDEX_MERGER_V9.persist( index, new File(tmpDir, String.valueOf(i)), - new IndexSpec() + new IndexSpec(), + null ); queryableIndexes.add(INDEX_IO.loadIndex(file)); diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java index ac41c687457..cfb40ceabd5 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java @@ -75,7 +75,7 @@ public class LongCompressionBenchmark File compFile = new File(dir, file + "-" + strategy + "-" + format); rand = new Random(); ByteBuffer buffer = Files.map(compFile); - supplier = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder(), null); + supplier = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder()); } @Benchmark diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java index bd00dfa62c0..2b654e72599 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java @@ -20,24 +20,20 @@ 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.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.LongSupplierSerializer; -import io.druid.segment.data.TmpFileIOPeon; import java.io.BufferedReader; -import java.io.ByteArrayOutputStream; import java.io.File; import java.io.IOException; -import java.io.OutputStream; import java.io.Writer; import java.net.URISyntaxException; -import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.FileChannel; import java.nio.charset.StandardCharsets; @@ -51,9 +47,10 @@ public class LongCompressionBenchmarkFileGenerator { private static final Logger log = new Logger(LongCompressionBenchmarkFileGenerator.class); public static final int ROW_NUM = 5000000; - public static final List compressions = - ImmutableList.of(CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.NONE); + public static final List compressions = + ImmutableList.of( + CompressionStrategy.LZ4, + CompressionStrategy.NONE); public static final List encodings = ImmutableList.of(CompressionFactory.LongEncodingStrategy.AUTO, CompressionFactory.LongEncodingStrategy.LONGS); @@ -134,7 +131,7 @@ public class LongCompressionBenchmarkFileGenerator // create compressed files using all combinations of CompressionStrategy and LongEncoding provided for (Map.Entry entry : generators.entrySet()) { - for (CompressedObjectStrategy.CompressionStrategy compression : compressions) { + for (CompressionStrategy compression : compressions) { for (CompressionFactory.LongEncodingStrategy encoding : encodings) { String name = entry.getKey() + "-" + compression.toString() + "-" + encoding.toString(); log.info("%s: ", name); @@ -142,42 +139,24 @@ public class LongCompressionBenchmarkFileGenerator compFile.delete(); File dataFile = new File(dir, entry.getKey()); - TmpFileIOPeon iopeon = new TmpFileIOPeon(true); LongSupplierSerializer writer = CompressionFactory.getLongSerializer( - iopeon, + new OffHeapMemorySegmentWriteOutMedium(), "long", ByteOrder.nativeOrder(), encoding, compression ); - BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8); - - try (FileChannel output = FileChannel.open( - compFile.toPath(), - StandardOpenOption.CREATE_NEW, - StandardOpenOption.WRITE - )) { + try ( + BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8); + 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.close(); - br.close(); + writer.writeTo(output, null); } log.info("%d", compFile.length() / 1024); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java index 52b6e468d55..71645ddcb2e 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -37,6 +37,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -136,6 +137,7 @@ public class TopNTypeInterfaceBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -145,7 +147,7 @@ public class TopNTypeInterfaceBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -291,7 +293,8 @@ public class TopNTypeInterfaceBenchmark File indexFile = INDEX_MERGER_V9.persist( incIndexes.get(i), tmpFile, - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); diff --git a/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java index 62e87cb851f..e6bca29f0f6 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java @@ -33,6 +33,7 @@ import io.druid.hll.HyperLogLogHash; import io.druid.java.util.common.ISE; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexBuilder; @@ -144,8 +145,8 @@ public class SegmentGenerator implements Closeable return Iterables.getOnlyElement(indexes); } else { try { - final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex( - TestHelper.getTestIndexMergerV9().merge( + final QueryableIndex merged = TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()).loadIndex( + TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()).merge( indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()), false, schemaInfo.getAggs() @@ -186,7 +187,7 @@ public class SegmentGenerator implements Closeable .create() .schema(indexSchema) .tmpDir(new File(new File(tempDir, identifier), String.valueOf(indexNumber))) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .rows(rows) .buildMMappedIndex(); } diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java index 5a96ef3785f..17ebafff409 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -39,6 +39,7 @@ import io.druid.segment.column.ColumnConfig; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.serde.ComplexMetrics; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.commons.io.FileUtils; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -95,6 +96,7 @@ public class IndexMergeBenchmark JSON_MAPPER.setInjectableValues(injectableValues); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -104,7 +106,7 @@ public class IndexMergeBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup @@ -144,7 +146,8 @@ public class IndexMergeBenchmark File indexFile = INDEX_MERGER_V9.persist( incIndex, tmpDir, - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); @@ -188,7 +191,8 @@ public class IndexMergeBenchmark rollup, schemaInfo.getAggsArray(), tmpFile, - new IndexSpec() + new IndexSpec(), + null ); blackhole.consume(mergedFile); diff --git a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java index bde2eda5ef5..1208b7864cc 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -28,6 +28,7 @@ import io.druid.data.input.InputRow; import io.druid.hll.HyperLogLogHash; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; @@ -88,6 +89,7 @@ public class IndexPersistBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -97,7 +99,7 @@ public class IndexPersistBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup @@ -172,7 +174,8 @@ public class IndexPersistBenchmark File indexFile = INDEX_MERGER_V9.persist( incIndex, tmpDir, - new IndexSpec() + new IndexSpec(), + null ); blackhole.consume(indexFile); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java index 94c576eae9c..ad8041d3dfa 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -47,6 +47,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -158,6 +159,7 @@ public class GroupByBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -167,7 +169,7 @@ public class GroupByBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -434,7 +436,8 @@ public class GroupByBenchmark final File file = INDEX_MERGER_V9.persist( index, new File(tmpDir, String.valueOf(i)), - new IndexSpec() + new IndexSpec(), + null ); queryableIndexes.add(INDEX_IO.loadIndex(file)); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java index be7476eba84..8b071f94bd2 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java @@ -38,6 +38,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.Druids.SearchQueryBuilder; import io.druid.query.FinalizeResultsQueryRunner; @@ -140,6 +141,7 @@ public class SearchBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -149,7 +151,7 @@ public class SearchBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -360,7 +362,8 @@ public class SearchBenchmark File indexFile = INDEX_MERGER_V9.persist( incIndexes.get(i), tmpDir, - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java index 271d6c45a95..410c3f5d83a 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java @@ -37,6 +37,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -134,6 +135,7 @@ public class SelectBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -143,7 +145,7 @@ public class SelectBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -223,7 +225,8 @@ public class SelectBenchmark File indexFile = INDEX_MERGER_V9.persist( incIndexes.get(i), tmpDir, - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); qIndexes.add(qIndex); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java index 57242d28771..a7aa4eb2ea6 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java @@ -35,6 +35,7 @@ import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -132,6 +133,7 @@ public class TimeseriesBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -141,7 +143,7 @@ public class TimeseriesBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -286,7 +288,8 @@ public class TimeseriesBenchmark File indexFile = INDEX_MERGER_V9.persist( incIndexes.get(i), tmpDir, - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); diff --git a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java index 7c57715f9c4..f213c91808a 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java @@ -36,6 +36,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -133,6 +134,7 @@ public class TopNBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -142,7 +144,7 @@ public class TopNBenchmark } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -263,7 +265,8 @@ public class TopNBenchmark File indexFile = INDEX_MERGER_V9.persist( incIndexes.get(i), tmpDir, - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); diff --git a/common/src/main/java/io/druid/common/utils/ByteUtils.java b/common/src/main/java/io/druid/common/utils/ByteUtils.java new file mode 100644 index 00000000000..f377a628e2b --- /dev/null +++ b/common/src/main/java/io/druid/common/utils/ByteUtils.java @@ -0,0 +1,35 @@ +/* + * 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.common.utils; + +public final class ByteUtils +{ + + public static byte checkedCast(int value) + { + byte result = (byte) value; + if (result != value) { + throw new IllegalArgumentException("Out of range: " + value); + } + return result; + } + + private ByteUtils() {} +} diff --git a/common/src/main/java/io/druid/common/utils/SerializerUtils.java b/common/src/main/java/io/druid/common/utils/SerializerUtils.java index 9191391f44c..0ead88af076 100644 --- a/common/src/main/java/io/druid/common/utils/SerializerUtils.java +++ b/common/src/main/java/io/druid/common/utils/SerializerUtils.java @@ -24,6 +24,7 @@ import com.google.common.io.OutputSupplier; import com.google.common.primitives.Floats; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +import io.druid.io.Channels; import io.druid.java.util.common.StringUtils; import java.io.IOException; @@ -38,38 +39,6 @@ import java.util.List; public class SerializerUtils { - /** - * Writes the given long value into the given OutputStream in big-endian byte order, using the helperBuffer. Faster - * alternative to out.write(Longs.toByteArray(value)), more convenient (sometimes) than wrapping the OutputStream into - * {@link java.io.DataOutputStream}. - * - * @param helperBuffer a big-endian heap ByteBuffer with capacity of at least 8 - */ - public static void writeBigEndianLongToOutputStream(OutputStream out, long value, ByteBuffer helperBuffer) - throws IOException - { - if (helperBuffer.order() != ByteOrder.BIG_ENDIAN || !helperBuffer.hasArray()) { - throw new IllegalArgumentException("Expected writable, big-endian, heap byteBuffer"); - } - helperBuffer.putLong(0, value); - out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Longs.BYTES); - } - - /** - * Writes the given long value into the given OutputStream in the native byte order, using the helperBuffer. - * - * @param helperBuffer a heap ByteBuffer with capacity of at least 8, with the native byte order - */ - public static void writeNativeOrderedLongToOutputStream(OutputStream out, long value, ByteBuffer helperBuffer) - throws IOException - { - if (helperBuffer.order() != ByteOrder.nativeOrder() || !helperBuffer.hasArray()) { - throw new IllegalArgumentException("Expected writable heap byteBuffer with the native byte order"); - } - helperBuffer.putLong(0, value); - out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Longs.BYTES); - } - /** * Writes the given int value into the given OutputStream in big-endian byte order, using the helperBuffer. Faster * alternative to out.write(Ints.toByteArray(value)), more convenient (sometimes) than wrapping the OutputStream into @@ -87,21 +56,6 @@ public class SerializerUtils out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Ints.BYTES); } - /** - * Writes the given int value into the given OutputStream in the native byte order, using the given helperBuffer. - * - * @param helperBuffer a heap ByteBuffer with capacity of at least 4, with the native byte order - */ - public static void writeNativeOrderedIntToOutputStream(OutputStream out, int value, ByteBuffer helperBuffer) - throws IOException - { - if (helperBuffer.order() != ByteOrder.nativeOrder() || !helperBuffer.hasArray()) { - throw new IllegalArgumentException("Expected writable heap byteBuffer with the native byte order"); - } - helperBuffer.putInt(0, value); - out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Ints.BYTES); - } - public void writeString(T out, String name) throws IOException { byte[] nameBytes = StringUtils.toUtf8(name); @@ -120,10 +74,10 @@ public class SerializerUtils { byte[] nameBytes = StringUtils.toUtf8(name); writeInt(out, nameBytes.length); - out.write(ByteBuffer.wrap(nameBytes)); + Channels.writeFully(out, ByteBuffer.wrap(nameBytes)); } - public String readString(InputStream in) throws IOException + String readString(InputStream in) throws IOException { final int length = readInt(in); byte[] stringBytes = new byte[length]; @@ -144,12 +98,12 @@ public class SerializerUtils return bytes; } - public void writeStrings(OutputStream out, String[] names) throws IOException + void writeStrings(OutputStream out, String[] names) throws IOException { writeStrings(out, Arrays.asList(names)); } - public void writeStrings(OutputStream out, List names) throws IOException + private void writeStrings(OutputStream out, List names) throws IOException { writeInt(out, names.size()); @@ -158,7 +112,7 @@ public class SerializerUtils } } - public String[] readStrings(InputStream in) throws IOException + String[] readStrings(InputStream in) throws IOException { int length = readInt(in); @@ -171,7 +125,7 @@ public class SerializerUtils return retVal; } - public String[] readStrings(ByteBuffer in) throws IOException + String[] readStrings(ByteBuffer in) throws IOException { int length = in.getInt(); @@ -184,20 +138,20 @@ public class SerializerUtils return retVal; } - public void writeInt(OutputStream out, int intValue) throws IOException + private void writeInt(OutputStream out, int intValue) throws IOException { out.write(Ints.toByteArray(intValue)); } - public void writeInt(WritableByteChannel out, int intValue) throws IOException + private void writeInt(WritableByteChannel out, int intValue) throws IOException { final ByteBuffer buffer = ByteBuffer.allocate(Ints.BYTES); buffer.putInt(intValue); buffer.flip(); - out.write(buffer); + Channels.writeFully(out, buffer); } - public int readInt(InputStream in) throws IOException + private int readInt(InputStream in) throws IOException { byte[] intBytes = new byte[Ints.BYTES]; @@ -206,7 +160,7 @@ public class SerializerUtils return Ints.fromByteArray(intBytes); } - public void writeInts(OutputStream out, int[] ints) throws IOException + void writeInts(OutputStream out, int[] ints) throws IOException { writeInt(out, ints.length); @@ -215,7 +169,7 @@ public class SerializerUtils } } - public int[] readInts(InputStream in) throws IOException + int[] readInts(InputStream in) throws IOException { int size = readInt(in); @@ -227,7 +181,7 @@ public class SerializerUtils return retVal; } - public void writeLong(OutputStream out, long longValue) throws IOException + private void writeLong(OutputStream out, long longValue) throws IOException { out.write(Longs.toByteArray(longValue)); } @@ -237,10 +191,10 @@ public class SerializerUtils final ByteBuffer buffer = ByteBuffer.allocate(Longs.BYTES); buffer.putLong(longValue); buffer.flip(); - out.write(buffer); + Channels.writeFully(out, buffer); } - public long readLong(InputStream in) throws IOException + long readLong(InputStream in) throws IOException { byte[] longBytes = new byte[Longs.BYTES]; @@ -249,7 +203,7 @@ public class SerializerUtils return Longs.fromByteArray(longBytes); } - public void writeLongs(OutputStream out, long[] longs) throws IOException + void writeLongs(OutputStream out, long[] longs) throws IOException { writeInt(out, longs.length); @@ -258,7 +212,7 @@ public class SerializerUtils } } - public long[] readLongs(InputStream in) throws IOException + long[] readLongs(InputStream in) throws IOException { int size = readInt(in); @@ -275,20 +229,20 @@ public class SerializerUtils writeInt(out, Float.floatToRawIntBits(floatValue)); } - public void writeFloat(WritableByteChannel out, float floatValue) throws IOException + void writeFloat(WritableByteChannel out, float floatValue) throws IOException { final ByteBuffer buffer = ByteBuffer.allocate(Floats.BYTES); buffer.putFloat(floatValue); buffer.flip(); - out.write(buffer); + Channels.writeFully(out, buffer); } - public float readFloat(InputStream in) throws IOException + float readFloat(InputStream in) throws IOException { return Float.intBitsToFloat(readInt(in)); } - public void writeFloats(OutputStream out, float[] floats) throws IOException + void writeFloats(OutputStream out, float[] floats) throws IOException { writeInt(out, floats.length); @@ -297,7 +251,7 @@ public class SerializerUtils } } - public float[] readFloats(InputStream in) throws IOException + float[] readFloats(InputStream in) throws IOException { int size = readInt(in); diff --git a/common/src/main/java/io/druid/io/ByteBufferInputStream.java b/common/src/main/java/io/druid/io/ByteBufferInputStream.java new file mode 100644 index 00000000000..189bf777e51 --- /dev/null +++ b/common/src/main/java/io/druid/io/ByteBufferInputStream.java @@ -0,0 +1,71 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +/** + * Streams bytes from the {@link ByteBuffer}'s position to the limit as {@link InputStream}. + */ +public final class ByteBufferInputStream extends InputStream +{ + private final ByteBuffer buffer; + + /** + * Does *not* make a copy of the given buffer, so the position of the given buffer is incremented, as the created + * InputStream is used. + */ + public ByteBufferInputStream(ByteBuffer buffer) + { + this.buffer = buffer; + } + + @Override + public int read() + { + if (!buffer.hasRemaining()) { + return -1; + } + return buffer.get() & 0xFF; + } + + @Override + public int read(byte[] bytes, int off, int len) + { + if (len == 0) { + return 0; + } + if (!buffer.hasRemaining()) { + return -1; + } + + len = Math.min(len, buffer.remaining()); + buffer.get(bytes, off, len); + return len; + } + + @Override + public int available() throws IOException + { + return buffer.remaining(); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/IOPeon.java b/common/src/main/java/io/druid/io/Channels.java similarity index 70% rename from processing/src/main/java/io/druid/segment/data/IOPeon.java rename to common/src/main/java/io/druid/io/Channels.java index 85a1b54e572..141da64d0fc 100644 --- a/processing/src/main/java/io/druid/segment/data/IOPeon.java +++ b/common/src/main/java/io/druid/io/Channels.java @@ -17,19 +17,23 @@ * under the License. */ -package io.druid.segment.data; +package io.druid.io; -import java.io.Closeable; -import java.io.File; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; -/** - */ -public interface IOPeon extends Closeable +public final class Channels { - OutputStream makeOutputStream(String filename) throws IOException; - InputStream makeInputStream(String filename) throws IOException; - File getFile(String filename); + + public static void writeFully(WritableByteChannel channel, ByteBuffer src) throws IOException + { + while (src.remaining() > 0) { + channel.write(src); + } + } + + private Channels() + { + } } diff --git a/docs/content/configuration/indexing-service.md b/docs/content/configuration/indexing-service.md index 4181fecfaaf..1ce3152ef58 100644 --- a/docs/content/configuration/indexing-service.md +++ b/docs/content/configuration/indexing-service.md @@ -343,3 +343,26 @@ If the peon is running in remote mode, there must be an overlord up and running. |`druid.peon.taskActionClient.retry.minWait`|The minimum retry time to communicate with overlord.|PT5S| |`druid.peon.taskActionClient.retry.maxWait`|The maximum retry time to communicate with overlord.|PT1M| |`druid.peon.taskActionClient.retry.maxRetryCount`|The maximum number of retries to communicate with overlord.|60| + +##### SegmentWriteOutMediumFactory + +When new segments are created, Druid temporarily stores some pre-processed data in some buffers. Currently two types of +*medium* exist for those buffers: *temporary files* and *off-heap memory*. + +*Temporary files* (`tmpFile`) are stored under the task working directory (see `druid.indexer.task.baseTaskDir` +configuration above) and thus share it's mounting properies, e. g. they could be backed by HDD, SSD or memory (tmpfs). +This type of medium may do unnecessary disk I/O and requires some disk space to be available. + +*Off-heap memory medium* (`offHeapMemory`) creates buffers in off-heap memory of a JVM process that is running a task. +This type of medium is preferred, but it may require to allow the JVM to have more off-heap memory, by changing +`-XX:MaxDirectMemorySize` configuration. It is not yet understood how does the required off-heap memory size relates +to the size of the segments being created. But definitely it doesn't make sense to add more extra off-heap memory, +than the configured maximum *heap* size (`-Xmx`) for the same JVM. + +For most types of tasks SegmentWriteOutMediumFactory could be configured per-task (see [Tasks](../ingestion/tasks.html) +page, "TuningConfig" section), but if it's not specified for a task, or it's not supported for a particular task type, +then the value from the configuration below is used: + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.peon.defaultSegmentWriteOutMediumFactory`|`tmpFile` or `offHeapMemory`, see explanation above|`tmpFile`| diff --git a/docs/content/development/extensions-core/kafka-ingestion.md b/docs/content/development/extensions-core/kafka-ingestion.md index f8c64ef823a..b32c479fe7b 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -129,6 +129,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)| |`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)| |`offsetFetchPeriod`|ISO8601 Period|How often the supervisor queries Kafka and the indexing tasks to fetch current offsets and calculate lag.|no (default == PT30S, min == PT5S)| +|`segmentWriteOutMediumFactory`|String|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used)| #### IndexSpec diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index 41d19727aa0..48e011ec046 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -154,6 +154,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |reportParseExceptions|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion. If false, unparseable rows and fields will be skipped. If an entire row is skipped, the "unparseable" counter will be incremented. If some fields in a row were parseable and some were not, the parseable fields will be indexed and the "unparseable" counter will not be incremented.|no (default == false)| |handoffConditionTimeout|long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.|no (default == 0)| |alertTimeout|long|Milliseconds timeout after which an alert is created if the task isn't finished by then. This allows users to monitor tasks that are failing to finish and give up the worker slot for any unexpected errors.|no (default == 0)| +|segmentWriteOutMediumFactory|String|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used)| |indexSpec|Object|Tune how data is indexed. See below for more information.|no| Before enabling thread priority settings, users are highly encouraged to read the [original pull request](https://github.com/druid-io/druid/pull/984) and other documentation about proper use of `-XX:+UseThreadPriorities`. diff --git a/docs/content/ingestion/tasks.md b/docs/content/ingestion/tasks.md index 628e2c9b976..b5b1bc1c543 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -143,9 +143,10 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no| |maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no| |forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no| -|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](./design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| +|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no| |reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no| |publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no| +|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no| #### IndexSpec diff --git a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java index e0c1ba03bab..6be52eb405a 100644 --- a/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java +++ b/extensions-core/datasketches/src/main/java/io/druid/query/aggregation/datasketches/theta/SketchMergeComplexMetricSerde.java @@ -21,10 +21,10 @@ package io.druid.query.aggregation.datasketches.theta; import com.yahoo.sketches.theta.Sketch; import io.druid.data.input.InputRow; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; @@ -80,9 +80,9 @@ public class SketchMergeComplexMetricSerde extends ComplexMetricSerde } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } } diff --git a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java index b1ad2f883b1..c5c469e1445 100644 --- a/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java +++ b/extensions-core/histogram/src/main/java/io/druid/query/aggregation/histogram/ApproximateHistogramFoldingSerde.java @@ -21,10 +21,10 @@ package io.druid.query.aggregation.histogram; import com.google.common.collect.Ordering; import io.druid.data.input.InputRow; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; @@ -102,9 +102,9 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } @Override diff --git a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index f74ab954136..45197c2c7e0 100644 --- a/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/io/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -39,7 +40,6 @@ import io.druid.query.filter.SelectorDimFilter; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.segment.IndexBuilder; import io.druid.segment.QueryableIndex; -import io.druid.segment.TestHelper; import io.druid.segment.column.ValueType; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.segment.virtual.ExpressionVirtualColumn; @@ -92,7 +92,7 @@ public class QuantileSqlAggregatorTest final QueryableIndex index = IndexBuilder.create() .tmpDir(temporaryFolder.newFolder()) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema( new IncrementalIndexSchema.Builder() .withMetrics( diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java index 6da2ef6a34c..5210028ded3 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaTuningConfig.java @@ -25,9 +25,12 @@ import io.druid.segment.IndexSpec; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.TuningConfig; import io.druid.segment.realtime.appenderator.AppenderatorConfig; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.File; +import java.util.Objects; public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig { @@ -44,20 +47,23 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig private final boolean reportParseExceptions; private final long handoffConditionTimeout; private final boolean resetOffsetAutomatically; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public KafkaTuningConfig( - @JsonProperty("maxRowsInMemory") Integer maxRowsInMemory, - @JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment, - @JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod, - @JsonProperty("basePersistDirectory") File basePersistDirectory, - @JsonProperty("maxPendingPersists") Integer maxPendingPersists, - @JsonProperty("indexSpec") IndexSpec indexSpec, + @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, + @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, + @JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod, + @JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory, + @JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists, + @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, // This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12. - @JsonProperty("buildV9Directly") Boolean buildV9Directly, - @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, - @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, - @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically + @JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly, + @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, + @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, + @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { // Cannot be a static because default basePersistDirectory is unique per-instance @@ -80,6 +86,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig this.resetOffsetAutomatically = resetOffsetAutomatically == null ? DEFAULT_RESET_OFFSET_AUTOMATICALLY : resetOffsetAutomatically; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } public static KafkaTuningConfig copyOf(KafkaTuningConfig config) @@ -94,7 +101,8 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig true, config.reportParseExceptions, config.handoffConditionTimeout, - config.resetOffsetAutomatically + config.resetOffsetAutomatically, + config.segmentWriteOutMediumFactory ); } @@ -169,6 +177,14 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig return resetOffsetAutomatically; } + @Override + @JsonProperty + @Nullable + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + public KafkaTuningConfig withBasePersistDirectory(File dir) { return new KafkaTuningConfig( @@ -181,7 +197,8 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig true, reportParseExceptions, handoffConditionTimeout, - resetOffsetAutomatically + resetOffsetAutomatically, + segmentWriteOutMediumFactory ); } @@ -194,54 +211,34 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig if (o == null || getClass() != o.getClass()) { return false; } - KafkaTuningConfig that = (KafkaTuningConfig) o; - - if (maxRowsInMemory != that.maxRowsInMemory) { - return false; - } - if (maxRowsPerSegment != that.maxRowsPerSegment) { - return false; - } - if (maxPendingPersists != that.maxPendingPersists) { - return false; - } - if (reportParseExceptions != that.reportParseExceptions) { - return false; - } - if (handoffConditionTimeout != that.handoffConditionTimeout) { - return false; - } - if (resetOffsetAutomatically != that.resetOffsetAutomatically) { - return false; - } - if (intermediatePersistPeriod != null - ? !intermediatePersistPeriod.equals(that.intermediatePersistPeriod) - : that.intermediatePersistPeriod != null) { - return false; - } - if (basePersistDirectory != null - ? !basePersistDirectory.equals(that.basePersistDirectory) - : that.basePersistDirectory != null) { - return false; - } - return indexSpec != null ? indexSpec.equals(that.indexSpec) : that.indexSpec == null; - + return maxRowsInMemory == that.maxRowsInMemory && + maxRowsPerSegment == that.maxRowsPerSegment && + maxPendingPersists == that.maxPendingPersists && + reportParseExceptions == that.reportParseExceptions && + handoffConditionTimeout == that.handoffConditionTimeout && + resetOffsetAutomatically == that.resetOffsetAutomatically && + Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) && + Objects.equals(basePersistDirectory, that.basePersistDirectory) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); } @Override public int hashCode() { - int result = maxRowsInMemory; - result = 31 * result + maxRowsPerSegment; - result = 31 * result + (intermediatePersistPeriod != null ? intermediatePersistPeriod.hashCode() : 0); - result = 31 * result + (basePersistDirectory != null ? basePersistDirectory.hashCode() : 0); - result = 31 * result + maxPendingPersists; - result = 31 * result + (indexSpec != null ? indexSpec.hashCode() : 0); - result = 31 * result + (reportParseExceptions ? 1 : 0); - result = 31 * result + (int) (handoffConditionTimeout ^ (handoffConditionTimeout >>> 32)); - result = 31 * result + (resetOffsetAutomatically ? 1 : 0); - return result; + return Objects.hash( + maxRowsInMemory, + maxRowsPerSegment, + intermediatePersistPeriod, + basePersistDirectory, + maxPendingPersists, + indexSpec, + reportParseExceptions, + handoffConditionTimeout, + resetOffsetAutomatically, + segmentWriteOutMediumFactory + ); } @Override @@ -257,6 +254,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig ", reportParseExceptions=" + reportParseExceptions + ", handoffConditionTimeout=" + handoffConditionTimeout + ", resetOffsetAutomatically=" + resetOffsetAutomatically + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + '}'; } } diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java index 559428fc2e4..d11709ecc62 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorSpec.java @@ -88,6 +88,7 @@ public class KafkaSupervisorSpec implements SupervisorSpec null, null, null, + null, null ); this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig"); diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java index 587d1a300dd..8d043877c38 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTuningConfig.java @@ -21,10 +21,12 @@ package io.druid.indexing.kafka.supervisor; import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexing.kafka.KafkaTuningConfig; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import org.joda.time.Duration; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.File; public class KafkaSupervisorTuningConfig extends KafkaTuningConfig @@ -48,6 +50,7 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("workerThreads") Integer workerThreads, @JsonProperty("chatThreads") Integer chatThreads, @JsonProperty("chatRetries") Long chatRetries, @@ -66,7 +69,8 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig true, reportParseExceptions, handoffConditionTimeout, - resetOffsetAutomatically + resetOffsetAutomatically, + segmentWriteOutMediumFactory ); this.workerThreads = workerThreads; @@ -126,6 +130,7 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + ", workerThreads=" + workerThreads + ", chatThreads=" + chatThreads + ", chatRetries=" + chatRetries + diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index e33405bc652..9d94127d4de 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1629,7 +1629,8 @@ public class KafkaIndexTaskTest true, reportParseExceptions, handoffConditionTimeout, - resetOffsetAutomatically + resetOffsetAutomatically, + null ); final Map context = isIncrementalHandoffSupported ? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java index 3bd5a389100..386664e4c0d 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaTuningConfigTest.java @@ -111,6 +111,7 @@ public class KafkaTuningConfigTest true, true, 5L, + null, null ); KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java index 347813858c6..cd159edf273 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java @@ -194,6 +194,7 @@ public class KafkaSupervisorTest extends EasyMockSupport false, null, null, + null, numThreads, TEST_CHAT_THREADS, TEST_CHAT_RETRIES, diff --git a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceSerde.java b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceSerde.java index 1ed59fff48c..cc41bf73bfd 100644 --- a/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceSerde.java +++ b/extensions-core/stats/src/main/java/io/druid/query/aggregation/variance/VarianceSerde.java @@ -21,10 +21,10 @@ package io.druid.query.aggregation.variance; import com.google.common.collect.Ordering; import io.druid.data.input.InputRow; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; @@ -122,9 +122,9 @@ public class VarianceSerde extends ComplexMetricSerde } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java index 1e0bca668f5..c9111d00cd3 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -501,7 +501,7 @@ public class IndexGeneratorJob implements Jobby ) throws IOException { return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist( - index, interval, file, config.getIndexSpec(), progressIndicator + index, interval, file, config.getIndexSpec(), progressIndicator, null ); } @@ -514,7 +514,7 @@ public class IndexGeneratorJob implements Jobby { boolean rollup = config.getSchema().getDataSchema().getGranularitySpec().isRollup(); return HadoopDruidIndexerConfig.INDEX_MERGER_V9.mergeQueryableIndex( - indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator + indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator, null ); } diff --git a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java index d19fc4026fe..eff3c4f4e10 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/updater/HadoopConverterJob.java @@ -535,7 +535,8 @@ public class HadoopConverterJob inDir, outDir, config.getIndexSpec(), - JobHelper.progressIndicatorForContext(context) + JobHelper.progressIndicatorForContext(context), + null ); } catch (Exception e) { 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 a06df928116..844df90b095 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,8 +56,8 @@ import io.druid.query.aggregation.DoubleSumAggregatorFactory; 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.CompressionStrategy; import io.druid.segment.data.RoaringBitmapSerdeFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.UniformGranularitySpec; @@ -293,8 +293,8 @@ public class HadoopConverterJobTest DATASOURCE, interval, new IndexSpec(new RoaringBitmapSerdeFactory(null), - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.UNCOMPRESSED, CompressionFactory.LongEncodingStrategy.LONGS), oldSemgments, true, @@ -399,8 +399,8 @@ public class HadoopConverterJobTest DATASOURCE, interval, new IndexSpec(new RoaringBitmapSerdeFactory(null), - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.UNCOMPRESSED, CompressionFactory.LongEncodingStrategy.LONGS), oldSemgments, true, diff --git a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java index 84a8c8c0c0c..b6e2a26d88f 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/index/YeOldePlumberSchool.java @@ -182,7 +182,14 @@ public class YeOldePlumberSchool implements PlumberSchool } fileToUpload = new File(tmpSegmentDir, "merged"); - indexMergerV9.mergeQueryableIndex(indexes, schema.getGranularitySpec().isRollup(), schema.getAggregators(), fileToUpload, config.getIndexSpec()); + indexMergerV9.mergeQueryableIndex( + indexes, + schema.getGranularitySpec().isRollup(), + schema.getAggregators(), + fileToUpload, + config.getIndexSpec(), + config.getSegmentWriteOutMediumFactory() + ); } // Map merged segment so we can extract dimensions @@ -230,7 +237,8 @@ public class YeOldePlumberSchool implements PlumberSchool indexMergerV9.persist( indexToPersist.getIndex(), dirToPersist, - config.getIndexSpec() + config.getIndexSpec(), + config.getSegmentWriteOutMediumFactory() ); indexToPersist.swapSegment(null); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java index 97cd6f6d6b9..c181d2eb2fc 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/AppendTask.java @@ -29,6 +29,7 @@ import com.google.common.collect.Lists; import io.druid.indexing.common.TaskToolbox; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.guava.Comparators; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; @@ -63,10 +64,11 @@ public class AppendTask extends MergeTaskBase @JsonProperty("indexSpec") IndexSpec indexSpec, // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { - super(id, dataSource, segments, context); + super(id, dataSource, segments, segmentWriteOutMediumFactory, context); this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.aggregators = aggregators; } @@ -139,7 +141,8 @@ public class AppendTask extends MergeTaskBase adapters, aggregators == null ? null : aggregators.toArray(new AggregatorFactory[aggregators.size()]), outDir, - indexSpec + indexSpec, + getSegmentWriteOutMediumFactory() ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentBackwardsCompatibleTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentBackwardsCompatibleTask.java index 0a2b9d2a12b..ca252c16061 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentBackwardsCompatibleTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentBackwardsCompatibleTask.java @@ -21,10 +21,13 @@ package io.druid.indexing.common.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import io.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; + @Deprecated public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask { @@ -36,7 +39,8 @@ public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask @JsonProperty("segment") DataSegment segment, @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, - @JsonProperty("validate") Boolean validate + @JsonProperty("validate") Boolean validate, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { super( @@ -47,6 +51,7 @@ public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask indexSpec, force == null ? false : force, validate == null ? false : validate, + segmentWriteOutMediumFactory, null ); } @@ -60,10 +65,11 @@ public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask @JsonProperty("segment") DataSegment segment, @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, - @JsonProperty("validate") Boolean validate + @JsonProperty("validate") Boolean validate, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { - super(groupId, segment, indexSpec, force, validate, null); + super(groupId, segment, indexSpec, force, validate, segmentWriteOutMediumFactory, null); } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java index 5ab8251d7f5..4605995ee61 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/ConvertSegmentTask.java @@ -36,12 +36,14 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexSpec; import io.druid.segment.loading.SegmentLoadingException; import io.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.Collections; @@ -61,12 +63,6 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask private static final Logger log = new Logger(ConvertSegmentTask.class); - @JsonIgnore - private final DataSegment segment; - private final IndexSpec indexSpec; - private final boolean force; - private final boolean validate; - /** * Create a segment converter task to convert a segment to the most recent version including the specified indexSpec * @@ -84,11 +80,22 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask IndexSpec indexSpec, boolean force, boolean validate, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { final String id = makeId(dataSource, interval); - return new ConvertSegmentTask(id, dataSource, interval, null, indexSpec, force, validate, context); + return new ConvertSegmentTask( + id, + dataSource, + interval, + null, + indexSpec, + force, + validate, + segmentWriteOutMediumFactory, + context + ); } /** @@ -106,13 +113,24 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask IndexSpec indexSpec, boolean force, boolean validate, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { final Interval interval = segment.getInterval(); final String dataSource = segment.getDataSource(); final String id = makeId(dataSource, interval); - return new ConvertSegmentTask(id, dataSource, interval, segment, indexSpec, force, validate, context); + return new ConvertSegmentTask( + id, + dataSource, + interval, + segment, + indexSpec, + force, + validate, + segmentWriteOutMediumFactory, + context + ); } protected static String makeId(String dataSource, Interval interval) @@ -131,22 +149,41 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { final boolean isForce = force == null ? false : force; final boolean isValidate = validate == null ? true : validate; if (id == null) { if (segment == null) { - return create(dataSource, interval, indexSpec, isForce, isValidate, context); + return create(dataSource, interval, indexSpec, isForce, isValidate, segmentWriteOutMediumFactory, context); } else { - return create(segment, indexSpec, isForce, isValidate, context); + return create(segment, indexSpec, isForce, isValidate, segmentWriteOutMediumFactory, context); } } - return new ConvertSegmentTask(id, dataSource, interval, segment, indexSpec, isForce, isValidate, context); + return new ConvertSegmentTask( + id, + dataSource, + interval, + segment, + indexSpec, + isForce, + isValidate, + segmentWriteOutMediumFactory, + context + ); } - protected ConvertSegmentTask( + @JsonIgnore + private final DataSegment segment; + private final IndexSpec indexSpec; + private final boolean force; + private final boolean validate; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + ConvertSegmentTask( String id, String dataSource, Interval interval, @@ -154,6 +191,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask IndexSpec indexSpec, boolean force, boolean validate, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -162,6 +200,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.force = force; this.validate = validate; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @JsonProperty @@ -194,6 +233,13 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask return segment; } + @JsonProperty + @Nullable + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { @@ -261,7 +307,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask @Override public Task apply(DataSegment input) { - return new SubTask(groupId, input, indexSpec, force, validate, context); + return new SubTask(groupId, input, indexSpec, force, validate, segmentWriteOutMediumFactory, context); } } ); @@ -293,6 +339,8 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask private final IndexSpec indexSpec; private final boolean force; private final boolean validate; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public SubTask( @@ -301,6 +349,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { @@ -321,6 +370,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.force = force == null ? false : force; this.validate = validate == null ? true : validate; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @JsonProperty @@ -352,7 +402,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask { log.info("Subs are good! Italian BMT and Meatball are probably my favorite."); try { - convertSegment(toolbox, segment, indexSpec, force, validate); + convertSegment(toolbox); } catch (Exception e) { log.error(e, "Conversion failed."); @@ -360,48 +410,42 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask } return success(); } - } - private static void convertSegment( - TaskToolbox toolbox, - final DataSegment segment, - IndexSpec indexSpec, - boolean force, - boolean validate - ) - throws SegmentLoadingException, IOException - { - log.info("Converting segment[%s]", segment); - final TaskActionClient actionClient = toolbox.getTaskActionClient(); - final List currentSegments = actionClient.submit( - new SegmentListUsedAction(segment.getDataSource(), segment.getInterval(), null) - ); + private void convertSegment(TaskToolbox toolbox) throws SegmentLoadingException, IOException + { + log.info("Converting segment[%s]", segment); + final TaskActionClient actionClient = toolbox.getTaskActionClient(); + final List currentSegments = actionClient.submit( + new SegmentListUsedAction(segment.getDataSource(), segment.getInterval(), null) + ); - for (DataSegment currentSegment : currentSegments) { - final String version = currentSegment.getVersion(); - final Integer binaryVersion = currentSegment.getBinaryVersion(); + for (DataSegment currentSegment : currentSegments) { + final String version = currentSegment.getVersion(); + final Integer binaryVersion = currentSegment.getBinaryVersion(); - if (!force && (version.startsWith(segment.getVersion()) && CURR_VERSION_INTEGER.equals(binaryVersion))) { - log.info("Skipping already updated segment[%s].", segment); - return; + if (!force && (version.startsWith(segment.getVersion()) && CURR_VERSION_INTEGER.equals(binaryVersion))) { + log.info("Skipping already updated segment[%s].", segment); + return; + } } - } - final Map localSegments = toolbox.fetchSegments(Collections.singletonList(segment)); + final Map localSegments = toolbox.fetchSegments(Collections.singletonList(segment)); - final File location = localSegments.get(segment); - final File outLocation = new File(location, "v9_out"); - if (toolbox.getIndexIO().convertSegment(location, outLocation, indexSpec, force, validate)) { - final int outVersion = IndexIO.getVersionFromDir(outLocation); + final File location = localSegments.get(segment); + final File outLocation = new File(location, "v9_out"); + IndexIO indexIO = toolbox.getIndexIO(); + if (indexIO.convertSegment(location, outLocation, indexSpec, force, validate, segmentWriteOutMediumFactory)) { + final int outVersion = IndexIO.getVersionFromDir(outLocation); - // Appending to the version makes a new version that inherits most comparability parameters of the original - // version, but is "newer" than said original version. - DataSegment updatedSegment = segment.withVersion(StringUtils.format("%s_v%s", segment.getVersion(), outVersion)); - updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); + // Appending to the version makes a new version that inherits most comparability parameters of the original + // version, but is "newer" than said original version. + DataSegment updatedSegment = segment.withVersion(StringUtils.format("%s_v%s", segment.getVersion(), outVersion)); + updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment); - actionClient.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); - } else { - log.info("Conversion failed."); + actionClient.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment))); + } else { + log.info("Conversion failed."); + } } } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java index 2a8e5a6b148..c06455d7601 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/HadoopConverterTask.java @@ -34,10 +34,12 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.java.util.common.UOE; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import io.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.IOException; import java.net.URI; import java.util.Arrays; @@ -52,6 +54,12 @@ public class HadoopConverterTask extends ConvertSegmentTask private static final String TYPE = "hadoop_convert_segment"; private static final Logger log = new Logger(HadoopConverterTask.class); + private final List hadoopDependencyCoordinates; + private final URI distributedSuccessCache; + private final String jobPriority; + private final String segmentOutputPath; + private final String classpathPrefix; + @JsonCreator public HadoopConverterTask( @JsonProperty("id") String id, @@ -65,6 +73,7 @@ public class HadoopConverterTask extends ConvertSegmentTask @JsonProperty("jobPriority") String jobPriority, @JsonProperty("segmentOutputPath") String segmentOutputPath, @JsonProperty("classpathPrefix") String classpathPrefix, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { @@ -81,6 +90,7 @@ public class HadoopConverterTask extends ConvertSegmentTask indexSpec, force, validate == null ? true : validate, + segmentWriteOutMediumFactory, context ); this.hadoopDependencyCoordinates = hadoopDependencyCoordinates; @@ -90,12 +100,6 @@ public class HadoopConverterTask extends ConvertSegmentTask this.classpathPrefix = classpathPrefix; } - private final List hadoopDependencyCoordinates; - private final URI distributedSuccessCache; - private final String jobPriority; - private final String segmentOutputPath; - private final String classpathPrefix; - @JsonProperty public List getHadoopDependencyCoordinates() { diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java index 93f6f105c35..1a9bade78c9 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/IndexTask.java @@ -57,6 +57,7 @@ import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.ParseException; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.DruidMetrics; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; @@ -115,6 +116,22 @@ public class IndexTask extends AbstractTask private static final HashFunction hashFunction = Hashing.murmur3_128(); private static final String TYPE = "index"; + private static String makeGroupId(IndexIngestionSpec ingestionSchema) + { + return makeGroupId(ingestionSchema.ioConfig.appendToExisting, ingestionSchema.dataSchema.getDataSource()); + } + + private static String makeGroupId(boolean isAppendToExisting, String dataSource) + { + if (isAppendToExisting) { + // Shared locking group for all tasks that append, since they are OK to run concurrently. + return StringUtils.format("%s_append_%s", TYPE, dataSource); + } else { + // Return null, one locking group per task. + return null; + } + } + @JsonIgnore private final IndexIngestionSpec ingestionSchema; @@ -156,22 +173,6 @@ public class IndexTask extends AbstractTask this.ingestionSchema = ingestionSchema; } - private static String makeGroupId(IndexIngestionSpec ingestionSchema) - { - return makeGroupId(ingestionSchema.ioConfig.appendToExisting, ingestionSchema.dataSchema.getDataSource()); - } - - private static String makeGroupId(boolean isAppendToExisting, String dataSource) - { - if (isAppendToExisting) { - // Shared locking group for all tasks that append, since they are OK to run concurrently. - return StringUtils.format("%s_append_%s", TYPE, dataSource); - } else { - // Return null, one locking group per task. - return null; - } - } - @Override public int getPriority() { @@ -964,6 +965,8 @@ public class IndexTask extends AbstractTask private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; private final long publishTimeout; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public IndexTuningConfig( @@ -979,7 +982,8 @@ public class IndexTask extends AbstractTask @JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs, @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, - @JsonProperty("publishTimeout") @Nullable Long publishTimeout + @JsonProperty("publishTimeout") @Nullable Long publishTimeout, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { this( @@ -993,13 +997,14 @@ public class IndexTask extends AbstractTask forceGuaranteedRollup, reportParseExceptions, publishTimeout, - null + null, + segmentWriteOutMediumFactory ); } private IndexTuningConfig() { - this(null, null, null, null, null, null, null, null, null, null, null); + this(null, null, null, null, null, null, null, null, null, null, null, null); } private IndexTuningConfig( @@ -1013,7 +1018,8 @@ public class IndexTask extends AbstractTask @Nullable Boolean forceGuaranteedRollup, @Nullable Boolean reportParseExceptions, @Nullable Long publishTimeout, - @Nullable File basePersistDirectory + @Nullable File basePersistDirectory, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { Preconditions.checkArgument( @@ -1047,6 +1053,8 @@ public class IndexTask extends AbstractTask !(this.forceExtendableShardSpecs && this.forceGuaranteedRollup), "Perfect rollup cannot be guaranteed with extendable shardSpecs" ); + + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } public IndexTuningConfig withBasePersistDirectory(File dir) @@ -1062,7 +1070,8 @@ public class IndexTask extends AbstractTask forceGuaranteedRollup, reportParseExceptions, publishTimeout, - dir + dir, + segmentWriteOutMediumFactory ); } @@ -1152,64 +1161,36 @@ public class IndexTask extends AbstractTask return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs } + @Nullable + @Override + @JsonProperty + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + @Override public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { return false; } - - final IndexTuningConfig that = (IndexTuningConfig) o; - - if (!Objects.equals(targetPartitionSize, that.targetPartitionSize)) { - return false; - } - - if (maxRowsInMemory != that.maxRowsInMemory) { - return false; - } - - if (maxTotalRows != that.maxTotalRows) { - return false; - } - - if (!Objects.equals(numShards, that.numShards)) { - return false; - } - - if (!Objects.equals(indexSpec, that.indexSpec)) { - return false; - } - - if (!Objects.equals(basePersistDirectory, that.basePersistDirectory)) { - return false; - } - - if (maxPendingPersists != that.maxPendingPersists) { - return false; - } - - if (forceExtendableShardSpecs != that.forceExtendableShardSpecs) { - return false; - } - - if (forceGuaranteedRollup != that.forceGuaranteedRollup) { - return false; - } - - if (reportParseExceptions != that.reportParseExceptions) { - return false; - } - - if (publishTimeout != that.publishTimeout) { - return false; - } - - return true; + IndexTuningConfig that = (IndexTuningConfig) o; + return maxRowsInMemory == that.maxRowsInMemory && + maxTotalRows == that.maxTotalRows && + maxPendingPersists == that.maxPendingPersists && + forceExtendableShardSpecs == that.forceExtendableShardSpecs && + forceGuaranteedRollup == that.forceGuaranteedRollup && + reportParseExceptions == that.reportParseExceptions && + publishTimeout == that.publishTimeout && + Objects.equals(targetPartitionSize, that.targetPartitionSize) && + Objects.equals(numShards, that.numShards) && + Objects.equals(indexSpec, that.indexSpec) && + Objects.equals(basePersistDirectory, that.basePersistDirectory) && + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); } @Override @@ -1226,7 +1207,8 @@ public class IndexTask extends AbstractTask forceExtendableShardSpecs, forceGuaranteedRollup, reportParseExceptions, - publishTimeout + publishTimeout, + segmentWriteOutMediumFactory ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java index d897df5c089..fe9ca37123c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTask.java @@ -28,6 +28,7 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.indexing.common.TaskToolbox; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; @@ -58,10 +59,11 @@ public class MergeTask extends MergeTaskBase @JsonProperty("indexSpec") IndexSpec indexSpec, // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { - super(id, dataSource, segments, context); + super(id, dataSource, segments, segmentWriteOutMediumFactory, context); this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations"); this.rollup = rollup == null ? Boolean.TRUE : rollup; this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; @@ -92,7 +94,8 @@ public class MergeTask extends MergeTaskBase rollup, aggregators.toArray(new AggregatorFactory[aggregators.size()]), outDir, - indexSpec + indexSpec, + getSegmentWriteOutMediumFactory() ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java index 5efab3f0abb..8a6cb91dea5 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/MergeTaskBase.java @@ -45,6 +45,7 @@ import io.druid.indexing.common.actions.TaskActionClient; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexIO; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -61,15 +62,19 @@ import java.util.Set; */ public abstract class MergeTaskBase extends AbstractFixedIntervalTask { + private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class); + @JsonIgnore private final List segments; - - private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class); + @JsonIgnore + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; protected MergeTaskBase( final String id, final String dataSource, final List segments, + final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -104,6 +109,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask verifyInputSegments(segments); this.segments = segments; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } protected void verifyInputSegments(List segments) @@ -254,6 +260,13 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask return segments; } + @JsonProperty + @Nullable + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + @Override public String toString() { @@ -262,6 +275,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask .add("dataSource", getDataSource()) .add("interval", getInterval()) .add("segments", segments) + .add("segmentWriteOutMediumFactory", segmentWriteOutMediumFactory) .toString(); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java index d5f59af6a3a..cdb60be12f6 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/SameIntervalMergeTask.java @@ -26,11 +26,13 @@ import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUsedAction; import io.druid.java.util.common.DateTimes; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexSpec; import io.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.List; import java.util.Map; @@ -43,6 +45,8 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask private final List aggregators; private final Boolean rollup; private final IndexSpec indexSpec; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; public SameIntervalMergeTask( @JsonProperty("id") String id, @@ -54,6 +58,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask // This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12. @SuppressWarnings("unused") @JsonProperty("buildV9Directly") Boolean buildV9Directly, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { @@ -66,6 +71,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations"); this.rollup = rollup == null ? Boolean.TRUE : rollup; this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @JsonProperty("aggregations") @@ -130,6 +136,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask aggregators, rollup, indexSpec, + segmentWriteOutMediumFactory, getContext() ); final TaskStatus status = mergeTask.run(toolbox); @@ -141,13 +148,14 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask public static class SubTask extends MergeTask { - public SubTask( + private SubTask( String baseId, String dataSource, List segments, List aggregators, Boolean rollup, IndexSpec indexSpec, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -159,6 +167,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask rollup, indexSpec, true, + segmentWriteOutMediumFactory, context ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java index f655f57225c..00c09435a9a 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TestUtils.java @@ -35,6 +35,7 @@ import io.druid.segment.IndexMergerV9; import io.druid.segment.column.ColumnConfig; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.segment.realtime.firehose.NoopChatHandlerProvider; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.server.security.AuthConfig; import io.druid.server.security.AuthorizerMapper; @@ -56,6 +57,7 @@ public class TestUtils this.jsonMapper = new DefaultObjectMapper(); indexIO = new IndexIO( jsonMapper, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -65,7 +67,7 @@ public class TestUtils } } ); - indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO); + indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); final List list = new ServerModule().getJacksonModules(); for (Module module : list) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java index c707979fbce..7bccdbab3d8 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/CompactionTaskTest.java @@ -69,7 +69,7 @@ import io.druid.segment.SimpleQueryableIndex; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.CompressionFactory.LongEncodingStrategy; import io.druid.segment.data.ListIndexed; import io.druid.segment.data.RoaringBitmapSerdeFactory; @@ -78,6 +78,7 @@ import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.transform.TransformingInputRowParser; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NumberedShardSpec; import org.hamcrest.CoreMatchers; @@ -240,7 +241,8 @@ public class CompactionTaskTest false, true, false, - 100L + 100L, + null ); } @@ -499,7 +501,7 @@ public class CompactionTaskTest indexIO, null, null, - new IndexMergerV9(objectMapper, indexIO), + new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), null, null, null, @@ -549,7 +551,7 @@ public class CompactionTaskTest Map segmentFileMap ) { - super(mapper, () -> 0); + super(mapper, OffHeapMemorySegmentWriteOutMediumFactory.instance(), () -> 0); queryableIndexMap = new HashMap<>(segmentFileMap.size()); for (Entry entry : segmentFileMap.entrySet()) { diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/ConvertSegmentTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/ConvertSegmentTaskTest.java index a512da94117..be94ec3bcbb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/ConvertSegmentTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/ConvertSegmentTaskTest.java @@ -51,7 +51,7 @@ public class ConvertSegmentTaskTest DateTime start = DateTimes.nowUtc(); final Interval interval = new Interval(start.minus(1000), start); - ConvertSegmentTask task = ConvertSegmentTask.create(dataSource, interval, null, false, true, null); + ConvertSegmentTask task = ConvertSegmentTask.create(dataSource, interval, null, false, true, null, null); Task task2 = jsonMapper.readValue(jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task), Task.class); Assert.assertEquals(task, task2); @@ -68,7 +68,7 @@ public class ConvertSegmentTaskTest 102937 ); - task = ConvertSegmentTask.create(segment, null, false, true, null); + task = ConvertSegmentTask.create(segment, null, false, true, null, null); task2 = jsonMapper.readValue(jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task), Task.class); Assert.assertEquals(task, task2); 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 be8885818c0..11a76b70098 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 @@ -25,8 +25,8 @@ import com.google.common.collect.ImmutableMap; import io.druid.indexing.common.TestUtils; import io.druid.java.util.common.Intervals; import io.druid.segment.IndexSpec; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -54,8 +54,8 @@ public class HadoopConverterTaskSerDeTest private static final int BINARY_VERSION = 34718; private static final long SEGMENT_SIZE = 7483901348790L; private static final IndexSpec INDEX_SPEC = new IndexSpec(new ConciseBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZF, + CompressionStrategy.LZ4, + CompressionStrategy.LZF, CompressionFactory.LongEncodingStrategy.LONGS); private static final DataSegment DATA_SEGMENT = new DataSegment( DATA_SOURCE, @@ -97,6 +97,7 @@ public class HadoopConverterTaskSerDeTest PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); final String strOrig = jsonMapper.writeValueAsString(orig); @@ -122,6 +123,7 @@ public class HadoopConverterTaskSerDeTest PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); HadoopConverterTask.ConverterSubTask subTask = new HadoopConverterTask.ConverterSubTask( @@ -175,6 +177,7 @@ public class HadoopConverterTaskSerDeTest PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); HadoopConverterTask.ConverterSubTask subTask = new HadoopConverterTask.ConverterSubTask( @@ -203,6 +206,7 @@ public class HadoopConverterTaskSerDeTest PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); Assert.assertTrue(orig.isValidate()); @@ -223,6 +227,7 @@ public class HadoopConverterTaskSerDeTest null, OUTPUT_PATH, null, + null, null ); Assert.assertEquals(DATA_SOURCE, parent.getDataSource()); @@ -248,6 +253,7 @@ public class HadoopConverterTaskSerDeTest PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); orig.getSegment(); @@ -268,6 +274,7 @@ public class HadoopConverterTaskSerDeTest null, OUTPUT_PATH, null, + null, null ); } @@ -287,6 +294,7 @@ public class HadoopConverterTaskSerDeTest null, OUTPUT_PATH, null, + null, null ); } @@ -306,6 +314,7 @@ public class HadoopConverterTaskSerDeTest null, OUTPUT_PATH, null, + null, null ); } @@ -325,6 +334,7 @@ public class HadoopConverterTaskSerDeTest null, null, null, + null, null ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index f9c85918b9f..79310c9771d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -1147,6 +1147,7 @@ public class IndexTaskTest forceExtendableShardSpecs, forceGuaranteedRollup, reportParseException, + null, null ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java index 0a61e0bdbbd..a51323803dc 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/MergeTaskBaseTest.java @@ -44,7 +44,7 @@ public class MergeTaskBaseTest .add(segmentBuilder.interval(Intervals.of("2012-01-03/2012-01-05")).build()) .build(); - final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments, null) + final MergeTaskBase testMergeTaskBase = new MergeTaskBase(null, "foo", segments, null, null) { @Override protected File merge(TaskToolbox toolbox, Map segments, File outDir) throws Exception diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index 1aa4db3ee88..bc0b0328ecb 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -931,6 +931,7 @@ public class RealtimeIndexTaskTest 0, reportParseExceptions, handoffTimeout, + null, null ); return new RealtimeIndexTask( diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java index bde1e9b12b0..63389a804ba 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java @@ -91,6 +91,7 @@ public class SameIntervalMergeTaskTest true, indexSpec, true, + null, null ); 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 83cdb0887c3..b0484c88223 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 @@ -35,12 +35,13 @@ import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import io.druid.indexing.common.task.IndexTask.IndexTuningConfig; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; 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.CompressionStrategy; import io.druid.segment.data.RoaringBitmapSerdeFactory; import io.druid.segment.indexing.DataSchema; import io.druid.segment.indexing.RealtimeIOConfig; @@ -189,7 +190,7 @@ public class TaskSerdeTest jsonMapper ), new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), - new IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null) + new IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null, null) ), null ); @@ -252,7 +253,7 @@ public class TaskSerdeTest jsonMapper ), new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) + new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null) ), null ); @@ -298,6 +299,7 @@ public class TaskSerdeTest true, indexSpec, true, + null, null ); @@ -347,6 +349,7 @@ public class TaskSerdeTest true, indexSpec, true, + null, null ); @@ -414,6 +417,7 @@ public class TaskSerdeTest null, false, true, + TmpFileSegmentWriteOutMediumFactory.instance(), null ); @@ -429,7 +433,8 @@ public class TaskSerdeTest Assert.assertEquals(task.getGroupId(), task2.getGroupId()); Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getInterval(), task2.getInterval()); - Assert.assertEquals(task.getSegment(), task.getSegment()); + Assert.assertEquals(task.getSegment(), task2.getSegment()); + Assert.assertEquals(task.getSegmentWriteOutMediumFactory(), task2.getSegmentWriteOutMediumFactory()); } @Test @@ -441,6 +446,7 @@ public class TaskSerdeTest indexSpec, false, true, + null, null ); @@ -503,6 +509,7 @@ public class TaskSerdeTest 0, true, null, + null, null ) ), @@ -567,6 +574,7 @@ public class TaskSerdeTest ), indexSpec, true, + null, null ); @@ -665,6 +673,7 @@ public class TaskSerdeTest indexSpec, false, true, + TmpFileSegmentWriteOutMediumFactory.instance(), null ); final String json = jsonMapper.writeValueAsString(task); @@ -686,41 +695,43 @@ public class TaskSerdeTest 0, 12345L ); - final ConvertSegmentTask convertSegmentTaskOriginal = ConvertSegmentTask.create( + final ConvertSegmentTask originalTask = ConvertSegmentTask.create( segment, new IndexSpec( new RoaringBitmapSerdeFactory(null), - CompressedObjectStrategy.CompressionStrategy.LZF, - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.LZF, + CompressionStrategy.UNCOMPRESSED, CompressionFactory.LongEncodingStrategy.LONGS ), false, true, + TmpFileSegmentWriteOutMediumFactory.instance(), null ); - final String json = jsonMapper.writeValueAsString(convertSegmentTaskOriginal); + final String json = jsonMapper.writeValueAsString(originalTask); final Task task = jsonMapper.readValue(json, Task.class); Assert.assertTrue(task instanceof ConvertSegmentTask); final ConvertSegmentTask convertSegmentTask = (ConvertSegmentTask) task; - Assert.assertEquals(convertSegmentTaskOriginal.getDataSource(), convertSegmentTask.getDataSource()); - Assert.assertEquals(convertSegmentTaskOriginal.getInterval(), convertSegmentTask.getInterval()); + Assert.assertEquals(originalTask.getDataSource(), convertSegmentTask.getDataSource()); + Assert.assertEquals(originalTask.getInterval(), convertSegmentTask.getInterval()); Assert.assertEquals( - convertSegmentTaskOriginal.getIndexSpec().getBitmapSerdeFactory().getClass().getCanonicalName(), + originalTask.getIndexSpec().getBitmapSerdeFactory().getClass().getCanonicalName(), convertSegmentTask.getIndexSpec() .getBitmapSerdeFactory() .getClass() .getCanonicalName() ); Assert.assertEquals( - convertSegmentTaskOriginal.getIndexSpec().getDimensionCompression(), + originalTask.getIndexSpec().getDimensionCompression(), convertSegmentTask.getIndexSpec().getDimensionCompression() ); Assert.assertEquals( - convertSegmentTaskOriginal.getIndexSpec().getMetricCompression(), + originalTask.getIndexSpec().getMetricCompression(), convertSegmentTask.getIndexSpec().getMetricCompression() ); Assert.assertEquals(false, convertSegmentTask.isForce()); Assert.assertEquals(segment, convertSegmentTask.getSegment()); + Assert.assertEquals(originalTask.getSegmentWriteOutMediumFactory(), convertSegmentTask.getSegmentWriteOutMediumFactory()); } @Test diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index d0fc7ac9bc1..30a84e1ca85 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -170,7 +170,7 @@ public class IngestSegmentFirehoseFactoryTest if (!persistDir.mkdirs() && !persistDir.exists()) { throw new IOE("Could not create directory at [%s]", persistDir.getAbsolutePath()); } - INDEX_MERGER_V9.persist(index, persistDir, indexSpec); + INDEX_MERGER_V9.persist(index, persistDir, indexSpec, null); final IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator(null, null, null) { diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index 912e741a57a..92a88939d8d 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -231,7 +231,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest } try { - INDEX_MERGER_V9.persist(index, persistDir, new IndexSpec()); + INDEX_MERGER_V9.persist(index, persistDir, new IndexSpec(), null); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 23cad066320..42b51ec5168 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -670,7 +670,7 @@ public class TaskLifecycleTest mapper ), new IndexIOConfig(new MockFirehoseFactory(false), false), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) + new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null) ), null ); @@ -728,7 +728,7 @@ public class TaskLifecycleTest mapper ), new IndexIOConfig(new MockExceptionalFirehoseFactory(), false), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null) + new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null) ), null ); @@ -1093,7 +1093,7 @@ public class TaskLifecycleTest mapper ), new IndexIOConfig(new MockFirehoseFactory(false), false), - new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null) + new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null, null) ), null ); @@ -1215,6 +1215,7 @@ public class TaskLifecycleTest 0, null, null, + null, null ); FireDepartment fireDepartment = new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig); diff --git a/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java b/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java index 4dd66dba882..2db064d9e6b 100644 --- a/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java +++ b/java-util/src/main/java/io/druid/java/util/common/io/smoosh/FileSmoosher.java @@ -46,7 +46,9 @@ import java.io.OutputStreamWriter; import java.io.Writer; import java.nio.ByteBuffer; import java.nio.channels.Channels; +import java.nio.channels.FileChannel; import java.nio.channels.GatheringByteChannel; +import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -291,21 +293,20 @@ public class FileSmoosher implements Closeable return new SmooshedWriter() { - private final FileOutputStream out = new FileOutputStream(tmpFile); - private final GatheringByteChannel channel = out.getChannel(); - private final Closer closer = Closer.create(); + private final GatheringByteChannel channel = + FileChannel.open( + tmpFile.toPath(), + StandardOpenOption.WRITE, + StandardOpenOption.CREATE, + StandardOpenOption.TRUNCATE_EXISTING + ); private int currOffset = 0; - { - closer.register(out); - closer.register(channel); - } - @Override public void close() throws IOException { - closer.close(); + channel.close(); completedFiles.add(tmpFile); filesInProcess.remove(tmpFile); diff --git a/pom.xml b/pom.xml index 5b10ff89314..00860684b67 100644 --- a/pom.xml +++ b/pom.xml @@ -227,7 +227,7 @@ com.ning compress-lzf - 1.0.3 + 1.0.4 io.airlift @@ -709,6 +709,8 @@ asm-commons 5.2 + + org.codehaus.jackson jackson-core-asl diff --git a/processing/pom.xml b/processing/pom.xml index 54963de0fb8..f1253cfdbce 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -35,6 +35,11 @@ druid-common ${project.parent.version} + + io.druid + java-util + ${project.parent.version} + io.druid druid-hll @@ -93,6 +98,10 @@ org.ow2.asm asm-commons + + it.unimi.dsi + fastutil + diff --git a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java index 26fda75b9aa..93b5aacea0a 100644 --- a/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java +++ b/processing/src/main/java/io/druid/query/aggregation/hyperloglog/HyperUniquesSerde.java @@ -23,10 +23,10 @@ import com.google.common.collect.Ordering; import io.druid.data.input.InputRow; import io.druid.hll.HyperLogLogCollector; import io.druid.hll.HyperLogLogHash; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexColumnPartSupplier; import io.druid.segment.serde.ComplexMetricExtractor; @@ -148,9 +148,9 @@ public class HyperUniquesSerde extends ComplexMetricSerde } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } } diff --git a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java index 8e1079cf63f..45043e18a47 100644 --- a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java @@ -19,23 +19,26 @@ package io.druid.segment; +import com.google.common.annotations.VisibleForTesting; +import io.druid.io.Channels; import io.druid.java.util.common.IAE; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIterable; import io.druid.segment.data.IndexedMultivalue; import io.druid.segment.data.WritableSupplier; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; -import java.util.ArrayList; import java.util.Iterator; -import java.util.List; /** * Format - @@ -55,7 +58,7 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier objectsIterable, - int maxValue, + final Iterable objectsIterable, + final int maxValue, final ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression, + final Closer closer ) { Iterator objects = objectsIterable.iterator(); - List offsetList = new ArrayList<>(); - List values = new ArrayList<>(); + IntList offsetList = new IntArrayList(); + IntList values = new IntArrayList(); int offset = 0; while (objects.hasNext()) { @@ -125,14 +128,16 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier objectsIterable, - int offsetChunkFactor, - int maxValue, + final Iterable objectsIterable, + final int offsetChunkFactor, + final int maxValue, final ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression, + final Closer closer ) { Iterator objects = objectsIterable.iterator(); - List offsetList = new ArrayList<>(); - List values = new ArrayList<>(); + IntArrayList offsetList = new IntArrayList(); + IntArrayList values = new IntArrayList(); int offset = 0; while (objects.hasNext()) { @@ -110,30 +106,32 @@ public class CompressedVSizeIndexedV3Supplier implements WritableSupplier makeMerger( IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) throws IOException; diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java index b17ac07b2f9..1fb6c53cefb 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSelector.java @@ -28,9 +28,6 @@ package io.druid.segment; */ public interface DoubleColumnSelector extends ColumnValueSelector { - @Override - double getDouble(); - /** * @deprecated This method is marked as deprecated in DoubleColumnSelector to minimize the probability of accidential * calling. "Polymorphism" of DoubleColumnSelector should be used only when operating on {@link ColumnValueSelector} diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java index 0e74e23990a..70979e84a65 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java @@ -19,13 +19,12 @@ package io.druid.segment; - import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.DoubleSupplierSerializer; -import io.druid.segment.data.IOPeon; import java.io.IOException; import java.nio.ByteOrder; @@ -34,28 +33,28 @@ import java.nio.channels.WritableByteChannel; public class DoubleColumnSerializer implements GenericColumnSerializer { public static DoubleColumnSerializer create( - IOPeon ioPeon, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - return new DoubleColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression); + return new DoubleColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression); } - private final IOPeon ioPeon; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ByteOrder byteOrder; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionStrategy compression; private DoubleSupplierSerializer writer; - public DoubleColumnSerializer( - IOPeon ioPeon, + private DoubleColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - this.ioPeon = ioPeon; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -65,7 +64,7 @@ public class DoubleColumnSerializer implements GenericColumnSerializer public void open() throws IOException { writer = CompressionFactory.getDoubleSerializer( - ioPeon, + segmentWriteOutMedium, StringUtils.format("%s.double_column", filenameBase), byteOrder, compression @@ -81,21 +80,15 @@ public class DoubleColumnSerializer implements GenericColumnSerializer } @Override - public void close() throws IOException - { - writer.close(); - } - - @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { return writer.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - writer.writeToChannel(channel, smoosher); + writer.writeTo(channel, smoosher); } } diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java b/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java index e37a70d0227..2f3e350852a 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java @@ -22,11 +22,10 @@ package io.druid.segment; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.GenericColumn; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.Closeable; -import java.io.File; import java.io.IOException; public class DoubleDimensionHandler implements DimensionHandler @@ -52,13 +51,16 @@ public class DoubleDimensionHandler implements DimensionHandler makeMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + ColumnCapabilities capabilities, + ProgressIndicator progress ) throws IOException { return new DoubleDimensionMergerV9( dimensionName, indexSpec, - ioPeon + segmentWriteOutMedium ); } diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java index 8a802350c05..3702df04312 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java @@ -21,9 +21,9 @@ package io.druid.segment; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.IOPeon; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.serde.DoubleGenericColumnPartSerde; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; import java.nio.IntBuffer; @@ -33,49 +33,32 @@ public class DoubleDimensionMergerV9 implements DimensionMergerV9 { protected String dimensionName; protected final IndexSpec indexSpec; - protected IOPeon ioPeon; private DoubleColumnSerializer serializer; public DoubleDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - IOPeon ioPeon + SegmentWriteOutMedium segmentWriteOutMedium ) { this.dimensionName = dimensionName; this.indexSpec = indexSpec; - this.ioPeon = ioPeon; try { - setupEncodedValueWriter(); + setupEncodedValueWriter(segmentWriteOutMedium); } catch (IOException ioe) { throw new RuntimeException(ioe); } } - protected void setupEncodedValueWriter() throws IOException + private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = DoubleColumnSerializer.create(ioPeon, dimensionName, metCompression); + final CompressionStrategy metCompression = indexSpec.getMetricCompression(); + this.serializer = DoubleColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression); serializer.open(); } - @Override - public ColumnDescriptor makeColumnDescriptor() throws IOException - { - serializer.close(); - final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); - builder.setValueType(ValueType.DOUBLE); - builder.addSerde( - DoubleGenericColumnPartSerde.serializerBuilder() - .withByteOrder(IndexIO.BYTE_ORDER) - .withDelegate(serializer) - .build() - ); - return builder.build(); - } - @Override public void writeMergedValueMetadata(List adapters) throws IOException { @@ -106,4 +89,18 @@ public class DoubleDimensionMergerV9 implements DimensionMergerV9 // a double column can never be all null return false; } + + @Override + public ColumnDescriptor makeColumnDescriptor() throws IOException + { + final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); + builder.setValueType(ValueType.DOUBLE); + builder.addSerde( + DoubleGenericColumnPartSerde.serializerBuilder() + .withByteOrder(IndexIO.BYTE_ORDER) + .withDelegate(serializer) + .build() + ); + return builder.build(); + } } diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSelector.java b/processing/src/main/java/io/druid/segment/FloatColumnSelector.java index 92b4a4c3195..0e890138936 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSelector.java @@ -28,9 +28,6 @@ package io.druid.segment; */ public interface FloatColumnSelector extends ColumnValueSelector { - @Override - float getFloat(); - /** * @deprecated This method is marked as deprecated in FloatColumnSelector to minimize the probability of accidential * calling. "Polymorphism" of FloatColumnSelector should be used only when operating on {@link ColumnValueSelector} diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java index 15560f14d85..cf39ddb85cd 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java @@ -21,10 +21,10 @@ package io.druid.segment; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.FloatSupplierSerializer; -import io.druid.segment.data.IOPeon; import java.io.IOException; import java.nio.ByteOrder; @@ -33,28 +33,28 @@ import java.nio.channels.WritableByteChannel; public class FloatColumnSerializer implements GenericColumnSerializer { public static FloatColumnSerializer create( - IOPeon ioPeon, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - return new FloatColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression); + return new FloatColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression); } - private final IOPeon ioPeon; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ByteOrder byteOrder; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionStrategy compression; private FloatSupplierSerializer writer; - public FloatColumnSerializer( - IOPeon ioPeon, + private FloatColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - this.ioPeon = ioPeon; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -64,7 +64,7 @@ public class FloatColumnSerializer implements GenericColumnSerializer public void open() throws IOException { writer = CompressionFactory.getFloatSerializer( - ioPeon, + segmentWriteOutMedium, StringUtils.format("%s.float_column", filenameBase), byteOrder, compression @@ -80,21 +80,14 @@ public class FloatColumnSerializer implements GenericColumnSerializer } @Override - public void close() throws IOException - { - writer.close(); - } - - @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { return writer.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - writer.writeToChannel(channel, smoosher); + writer.writeTo(channel, smoosher); } - } diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java index 99243913a6a..a946bf9e1bc 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java @@ -22,11 +22,10 @@ package io.druid.segment; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.GenericColumn; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.Closeable; -import java.io.File; import java.io.IOException; public class FloatDimensionHandler implements DimensionHandler @@ -52,13 +51,16 @@ public class FloatDimensionHandler implements DimensionHandler makeMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + ColumnCapabilities capabilities, + ProgressIndicator progress ) throws IOException { return new FloatDimensionMergerV9( dimensionName, indexSpec, - ioPeon + segmentWriteOutMedium ); } diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java index e204a4ded53..0a2070f4cf1 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java @@ -21,9 +21,9 @@ package io.druid.segment; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedObjectStrategy; -import io.druid.segment.data.IOPeon; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.serde.FloatGenericColumnPartSerde; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; import java.nio.IntBuffer; @@ -33,32 +33,29 @@ public class FloatDimensionMergerV9 implements DimensionMergerV9 { protected String dimensionName; protected final IndexSpec indexSpec; - protected IOPeon ioPeon; - private FloatColumnSerializer serializer; public FloatDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - IOPeon ioPeon + SegmentWriteOutMedium segmentWriteOutMedium ) { this.dimensionName = dimensionName; this.indexSpec = indexSpec; - this.ioPeon = ioPeon; try { - setupEncodedValueWriter(); + setupEncodedValueWriter(segmentWriteOutMedium); } catch (IOException ioe) { throw new RuntimeException(ioe); } } - protected void setupEncodedValueWriter() throws IOException + private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = FloatColumnSerializer.create(ioPeon, dimensionName, metCompression); + final CompressionStrategy metCompression = indexSpec.getMetricCompression(); + this.serializer = FloatColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression); serializer.open(); } @@ -96,7 +93,6 @@ public class FloatDimensionMergerV9 implements DimensionMergerV9 @Override public ColumnDescriptor makeColumnDescriptor() throws IOException { - serializer.close(); final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); builder.setValueType(ValueType.FLOAT); builder.addSerde( diff --git a/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java b/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java index a225383e40f..a9f35004fab 100644 --- a/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java @@ -20,20 +20,14 @@ package io.druid.segment; import io.druid.guice.annotations.ExtensionPoint; -import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.Serializer; -import java.io.Closeable; import java.io.IOException; -import java.nio.channels.WritableByteChannel; @ExtensionPoint -public interface GenericColumnSerializer extends Closeable +public interface GenericColumnSerializer extends Serializer { void open() throws IOException; void serialize(Object obj) throws IOException; - - long getSerializedSize(); - - void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 18dcaa6d7b7..60faadd177c 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -44,6 +44,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnCapabilities; @@ -70,6 +71,7 @@ import io.druid.segment.serde.LongGenericColumnSupplier; import io.druid.segment.serde.SpatialIndexColumnPartSupplier; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -95,11 +97,13 @@ public class IndexIO private static final SerializerUtils serializerUtils = new SerializerUtils(); private final ObjectMapper mapper; + private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory; @Inject - public IndexIO(ObjectMapper mapper, ColumnConfig columnConfig) + public IndexIO(ObjectMapper mapper, SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory, ColumnConfig columnConfig) { this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); + this.defaultSegmentWriteOutMediumFactory = Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory"); Preconditions.checkNotNull(columnConfig, "null ColumnConfig"); ImmutableMap.Builder indexLoadersBuilder = ImmutableMap.builder(); LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig); @@ -224,13 +228,17 @@ public class IndexIO File converted, IndexSpec indexSpec, boolean forceIfCurrent, - boolean validate + boolean validate, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { final int version = SegmentUtils.getVersionFromDir(toConvert); boolean current = version == CURRENT_VERSION_ID; if (!current || forceIfCurrent) { - new IndexMergerV9(mapper, this).convert(toConvert, converted, indexSpec); + if (segmentWriteOutMediumFactory == null) { + segmentWriteOutMediumFactory = this.defaultSegmentWriteOutMediumFactory; + } + new IndexMergerV9(mapper, this, segmentWriteOutMediumFactory).convert(toConvert, converted, indexSpec); if (validate) { validateTwoSegments(toConvert, converted); } @@ -341,8 +349,7 @@ public class IndexIO CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer( smooshedFiles.mapFile(makeTimeFile(inDir, BYTE_ORDER).getName()), - BYTE_ORDER, - smooshedFiles + BYTE_ORDER ); Map metrics = Maps.newLinkedHashMap(); diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 40ad0cdf805..858658987b1 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; @@ -39,6 +40,7 @@ import io.druid.java.util.common.parsers.CloseableIterator; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntSortedSet; @@ -156,7 +158,12 @@ public interface IndexMerger return Lists.newArrayList(retVal); } - File persist(IncrementalIndex index, File outDir, IndexSpec indexSpec) throws IOException; + File persist( + IncrementalIndex index, + File outDir, + IndexSpec indexSpec, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) throws IOException; /** * This is *not* thread-safe and havok will ensue if this is called and writes are still occurring @@ -170,22 +177,21 @@ public interface IndexMerger * * @throws IOException if an IO error occurs persisting the index */ - File persist(IncrementalIndex index, Interval dataInterval, File outDir, IndexSpec indexSpec) throws IOException; + File persist( + IncrementalIndex index, + Interval dataInterval, + File outDir, + IndexSpec indexSpec, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) throws IOException; File persist( IncrementalIndex index, Interval dataInterval, File outDir, IndexSpec indexSpec, - ProgressIndicator progress - ) throws IOException; - - File mergeQueryableIndex( - List indexes, - boolean rollup, - AggregatorFactory[] metricAggs, - File outDir, - IndexSpec indexSpec + ProgressIndicator progress, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; File mergeQueryableIndex( @@ -194,9 +200,20 @@ public interface IndexMerger AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; + File mergeQueryableIndex( + List indexes, + boolean rollup, + AggregatorFactory[] metricAggs, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) throws IOException; + + @VisibleForTesting File merge( List indexes, boolean rollup, @@ -208,11 +225,21 @@ public interface IndexMerger // Faster than IndexMaker File convert(File inDir, File outDir, IndexSpec indexSpec) throws IOException; - File convert(File inDir, File outDir, IndexSpec indexSpec, ProgressIndicator progress) - throws IOException; + File convert( + File inDir, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) throws IOException; - File append(List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec) - throws IOException; + File append( + List indexes, + AggregatorFactory[] aggregators, + File outDir, + IndexSpec indexSpec, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) throws IOException; interface IndexSeeker { @@ -385,7 +412,8 @@ public interface IndexMerger Int2ObjectMap.Entry entry = entryIterator.next(); for (IntIterator setIterator = entry.getValue().iterator(); setIterator.hasNext(); /* NOP */) { - retVal.addRow(entry.getIntKey(), setIterator.nextInt()); + int rowNum = setIterator.nextInt(); + retVal.addRow(entry.getIntKey(), rowNum); } } } diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index 609f6387d44..6d9be2ced61 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -28,17 +28,16 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; -import com.google.common.io.ByteStreams; import com.google.common.io.Files; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.inject.Inject; import io.druid.collections.CombiningIterable; -import io.druid.java.util.common.DateTimes; -import io.druid.java.util.common.JodaUtils; import io.druid.io.ZeroCopyByteArrayOutputStream; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.MergeIterable; @@ -52,12 +51,10 @@ import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; -import io.druid.segment.data.TmpFileIOPeon; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; import io.druid.segment.loading.MMappedQueryableSegmentizerFactory; @@ -67,6 +64,8 @@ import io.druid.segment.serde.ComplexMetrics; import io.druid.segment.serde.DoubleGenericColumnPartSerde; import io.druid.segment.serde.FloatGenericColumnPartSerde; import io.druid.segment.serde.LongGenericColumnPartSerde; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntSortedSet; @@ -75,7 +74,6 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; -import java.io.Closeable; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -91,30 +89,18 @@ import java.util.Set; public class IndexMergerV9 implements IndexMerger { private static final Logger log = new Logger(IndexMergerV9.class); - protected final ObjectMapper mapper; - protected final IndexIO indexIO; + + private final ObjectMapper mapper; + private final IndexIO indexIO; + private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory; @Inject - public IndexMergerV9( - ObjectMapper mapper, - IndexIO indexIO - ) + public IndexMergerV9(ObjectMapper mapper, IndexIO indexIO, SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory) { this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); - - } - - private static void registerDeleteDirectory(Closer closer, final File dir) - { - closer.register(new Closeable() - { - @Override - public void close() throws IOException - { - FileUtils.deleteDirectory(dir); - } - }); + this.defaultSegmentWriteOutMediumFactory = + Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory"); } private File makeIndexFiles( @@ -125,7 +111,8 @@ public class IndexMergerV9 implements IndexMerger final List mergedDimensions, final List mergedMetrics, final Function>, Iterable> rowMergerFn, - final IndexSpec indexSpec + final IndexSpec indexSpec, + final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { progress.start(); @@ -163,21 +150,15 @@ public class IndexMergerV9 implements IndexMerger Closer closer = Closer.create(); try { final FileSmoosher v9Smoosher = new FileSmoosher(outDir); - final File v9TmpDir = new File(outDir, "v9-tmp"); - FileUtils.forceMkdir(v9TmpDir); - registerDeleteDirectory(closer, v9TmpDir); - log.info("Start making v9 index files, outDir:%s", outDir); + FileUtils.forceMkdir(outDir); - File tmpPeonFilesDir = new File(v9TmpDir, "tmpPeonFiles"); - FileUtils.forceMkdir(tmpPeonFilesDir); - registerDeleteDirectory(closer, tmpPeonFilesDir); - final IOPeon ioPeon = new TmpFileIOPeon(tmpPeonFilesDir, false); - closer.register(ioPeon); + SegmentWriteOutMediumFactory omf = segmentWriteOutMediumFactory != null ? segmentWriteOutMediumFactory + : defaultSegmentWriteOutMediumFactory; + log.info("Using SegmentWriteOutMediumFactory[%s]", omf.getClass().getSimpleName()); + SegmentWriteOutMedium segmentWriteOutMedium = omf.makeSegmentWriteOutMedium(outDir); + closer.register(segmentWriteOutMedium); long startTime = System.currentTimeMillis(); - ByteStreams.write( - Ints.toByteArray(IndexIO.V9_VERSION), - Files.newOutputStreamSupplier(new File(outDir, "version.bin")) - ); + Files.asByteSink(new File(outDir, "version.bin")).write(Ints.toByteArray(IndexIO.V9_VERSION)); log.info("Completed version.bin in %,d millis.", System.currentTimeMillis() - startTime); progress.progress(); @@ -196,7 +177,7 @@ public class IndexMergerV9 implements IndexMerger final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); final List mergers = new ArrayList<>(); for (int i = 0; i < mergedDimensions.size(); i++) { - mergers.add(handlers[i].makeMerger(indexSpec, v9TmpDir, ioPeon, dimCapabilities.get(i), progress)); + mergers.add(handlers[i].makeMerger(indexSpec, segmentWriteOutMedium, dimCapabilities.get(i), progress)); } /************* Setup Dim Conversions **************/ @@ -215,15 +196,17 @@ public class IndexMergerV9 implements IndexMerger handlers, mergers ); - final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon, indexSpec); + final LongColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); final ArrayList metWriters = setupMetricsWriters( - ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec + segmentWriteOutMedium, + mergedMetrics, + metricsValueTypes, + metricTypeNames, + indexSpec ); final List rowNumConversions = Lists.newArrayListWithCapacity(adapters.size()); - mergeIndexesAndWriteColumns( - adapters, progress, theRows, timeWriter, metWriters, rowNumConversions, mergers - ); + mergeIndexesAndWriteColumns(adapters, progress, theRows, timeWriter, metWriters, rowNumConversions, mergers); /************ Create Inverted Indexes and Finalize Build Columns *************/ final String section = "build inverted index and columns"; @@ -245,9 +228,7 @@ public class IndexMergerV9 implements IndexMerger /************* Make index.drd & metadata.drd files **************/ progress.progress(); - makeIndexBinary( - v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, progress, indexSpec, mergers - ); + makeIndexBinary(v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, progress, indexSpec, mergers); makeMetadataBinary(v9Smoosher, progress, segmentMetadata); v9Smoosher.close(); @@ -312,8 +293,8 @@ public class IndexMergerV9 implements IndexMerger + serializerUtils.getSerializedStringByteSize(bitmapSerdeFactoryType); final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes); - cols.writeToChannel(writer); - dims.writeToChannel(writer); + cols.writeTo(writer, v9Smoosher); + dims.writeTo(writer, v9Smoosher); DateTime minTime = DateTimes.MAX; DateTime maxTime = DateTimes.MIN; @@ -327,9 +308,7 @@ public class IndexMergerV9 implements IndexMerger serializerUtils.writeLong(writer, dataInterval.getStartMillis()); serializerUtils.writeLong(writer, dataInterval.getEndMillis()); - serializerUtils.writeString( - writer, bitmapSerdeFactoryType - ); + serializerUtils.writeString(writer, bitmapSerdeFactoryType); writer.close(); IndexIO.checkFileSize(new File(outDir, "index.drd")); @@ -355,7 +334,6 @@ public class IndexMergerV9 implements IndexMerger String metric = mergedMetrics.get(i); long metricStartTime = System.currentTimeMillis(); GenericColumnSerializer writer = metWriters.get(i); - writer.close(); final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); ValueType type = metricsValueTypes.get(metric); @@ -421,8 +399,6 @@ public class IndexMergerV9 implements IndexMerger progress.startSection(section); long startTime = System.currentTimeMillis(); - timeWriter.close(); - final ColumnDescriptor serdeficator = ColumnDescriptor .builder() .setValueType(ValueType.LONG) @@ -447,10 +423,11 @@ public class IndexMergerV9 implements IndexMerger ZeroCopyByteArrayOutputStream specBytes = new ZeroCopyByteArrayOutputStream(); serializerUtils.writeString(specBytes, mapper.writeValueAsString(serdeficator)); try (SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( - columnName, serdeficator.numBytes() + specBytes.size() + columnName, + specBytes.size() + serdeficator.getSerializedSize() )) { specBytes.writeTo(channel); - serdeficator.write(channel, v9Smoosher); + serdeficator.writeTo(channel, v9Smoosher); } } @@ -521,10 +498,12 @@ public class IndexMergerV9 implements IndexMerger progress.stopSection(section); } - private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon, final IndexSpec indexSpec) throws IOException + private LongColumnSerializer setupTimeWriter(SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec) throws IOException { LongColumnSerializer timeWriter = LongColumnSerializer.create( - ioPeon, "little_end_time", CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY, + segmentWriteOutMedium, + "little_end_time", + CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY, indexSpec.getLongEncoding() ); // we will close this writer after we added all the timestamps @@ -533,7 +512,7 @@ public class IndexMergerV9 implements IndexMerger } private ArrayList setupMetricsWriters( - final IOPeon ioPeon, + final SegmentWriteOutMedium segmentWriteOutMedium, final List mergedMetrics, final Map metricsValueTypes, final Map metricTypeNames, @@ -541,20 +520,20 @@ public class IndexMergerV9 implements IndexMerger ) throws IOException { ArrayList metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size()); - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); + final 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, longEncoding); + writer = LongColumnSerializer.create(segmentWriteOutMedium, metric, metCompression, longEncoding); break; case FLOAT: - writer = FloatColumnSerializer.create(ioPeon, metric, metCompression); + writer = FloatColumnSerializer.create(segmentWriteOutMedium, metric, metCompression); break; case DOUBLE: - writer = DoubleColumnSerializer.create(ioPeon, metric, metCompression); + writer = DoubleColumnSerializer.create(segmentWriteOutMedium, metric, metCompression); break; case COMPLEX: final String typeName = metricTypeNames.get(metric); @@ -562,7 +541,7 @@ public class IndexMergerV9 implements IndexMerger if (serde == null) { throw new ISE("Unknown type[%s]", typeName); } - writer = serde.getSerializer(ioPeon, metric); + writer = serde.getSerializer(segmentWriteOutMedium, metric); break; default: throw new ISE("Unknown type[%s]", type); @@ -628,21 +607,11 @@ public class IndexMergerV9 implements IndexMerger public File persist( final IncrementalIndex index, File outDir, - IndexSpec indexSpec + IndexSpec indexSpec, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { - return persist(index, index.getInterval(), outDir, indexSpec); - } - - @Override - public File persist( - final IncrementalIndex index, - final Interval dataInterval, - File outDir, - IndexSpec indexSpec - ) throws IOException - { - return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator()); + return persist(index, index.getInterval(), outDir, indexSpec, segmentWriteOutMediumFactory); } @Override @@ -651,7 +620,20 @@ public class IndexMergerV9 implements IndexMerger final Interval dataInterval, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) throws IOException + { + return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator(), segmentWriteOutMediumFactory); + } + + @Override + public File persist( + final IncrementalIndex index, + final Interval dataInterval, + File outDir, + IndexSpec indexSpec, + ProgressIndicator progress, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { if (index.isEmpty()) { @@ -688,7 +670,8 @@ public class IndexMergerV9 implements IndexMerger index.getMetricAggs(), outDir, indexSpec, - progress + progress, + segmentWriteOutMediumFactory ); } @@ -698,10 +681,19 @@ public class IndexMergerV9 implements IndexMerger boolean rollup, final AggregatorFactory[] metricAggs, File outDir, - IndexSpec indexSpec + IndexSpec indexSpec, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { - return mergeQueryableIndex(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); + return mergeQueryableIndex( + indexes, + rollup, + metricAggs, + outDir, + indexSpec, + new BaseProgressIndicator(), + segmentWriteOutMediumFactory + ); } @Override @@ -711,7 +703,8 @@ public class IndexMergerV9 implements IndexMerger final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + ProgressIndicator progress, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { return merge( @@ -720,7 +713,8 @@ public class IndexMergerV9 implements IndexMerger metricAggs, outDir, indexSpec, - progress + progress, + segmentWriteOutMediumFactory ); } @@ -733,16 +727,17 @@ public class IndexMergerV9 implements IndexMerger IndexSpec indexSpec ) throws IOException { - return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); + return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator(), null); } - public File merge( + private File merge( List indexes, final boolean rollup, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + ProgressIndicator progress, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { FileUtils.deleteDirectory(outDir); @@ -847,19 +842,25 @@ public class IndexMergerV9 implements IndexMerger mergedDimensions, mergedMetrics, rowMergerFn, - indexSpec + indexSpec, + segmentWriteOutMediumFactory ); } @Override public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException { - return convert(inDir, outDir, indexSpec, new BaseProgressIndicator()); + return convert(inDir, outDir, indexSpec, new BaseProgressIndicator(), defaultSegmentWriteOutMediumFactory); } @Override - public File convert(final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress) - throws IOException + public File convert( + final File inDir, + final File outDir, + final IndexSpec indexSpec, + final ProgressIndicator progress, + final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory + ) throws IOException { try (QueryableIndex index = indexIO.loadIndex(inDir)) { final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); @@ -879,25 +880,19 @@ public class IndexMergerV9 implements IndexMerger return input.get(0); } }, - indexSpec + indexSpec, + segmentWriteOutMediumFactory ); } } @Override - public File append( - List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec - ) throws IOException - { - return append(indexes, aggregators, outDir, indexSpec, new BaseProgressIndicator()); - } - public File append( List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { FileUtils.deleteDirectory(outDir); @@ -944,11 +939,12 @@ public class IndexMergerV9 implements IndexMerger indexes, aggregators, outDir, - progress, + new BaseProgressIndicator(), mergedDimensions, mergedMetrics, rowMergerFn, - indexSpec + indexSpec, + segmentWriteOutMediumFactory ); } diff --git a/processing/src/main/java/io/druid/segment/IndexSpec.java b/processing/src/main/java/io/druid/segment/IndexSpec.java index 67ea64d2ac3..073a6a879af 100644 --- a/processing/src/main/java/io/druid/segment/IndexSpec.java +++ b/processing/src/main/java/io/druid/segment/IndexSpec.java @@ -25,8 +25,8 @@ import com.google.common.base.Preconditions; 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.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import java.util.Arrays; @@ -41,16 +41,16 @@ import java.util.Set; */ public class IndexSpec { - 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 CompressionStrategy DEFAULT_METRIC_COMPRESSION = CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY; + public static final CompressionStrategy DEFAULT_DIMENSION_COMPRESSION = CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY; public static final CompressionFactory.LongEncodingStrategy DEFAULT_LONG_ENCODING = CompressionFactory.DEFAULT_LONG_ENCODING_STRATEGY; - private static final Set METRIC_COMPRESSION = Sets.newHashSet( - Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values()) + private static final Set METRIC_COMPRESSION = Sets.newHashSet( + Arrays.asList(CompressionStrategy.values()) ); - private static final Set DIMENSION_COMPRESSION = Sets.newHashSet( - Arrays.asList(CompressedObjectStrategy.CompressionStrategy.noNoneValues()) + private static final Set DIMENSION_COMPRESSION = Sets.newHashSet( + Arrays.asList(CompressionStrategy.noNoneValues()) ); private static final Set LONG_ENCODING_NAMES = Sets.newHashSet( @@ -58,8 +58,8 @@ public class IndexSpec ); private final BitmapSerdeFactory bitmapSerdeFactory; - private final CompressedObjectStrategy.CompressionStrategy dimensionCompression; - private final CompressedObjectStrategy.CompressionStrategy metricCompression; + private final CompressionStrategy dimensionCompression; + private final CompressionStrategy metricCompression; private final CompressionFactory.LongEncodingStrategy longEncoding; @@ -80,10 +80,10 @@ public class IndexSpec * setting, or, if none was set, uses the default defined in {@link BitmapSerde} * * @param dimensionCompression compression format for dimension columns, null to use the default. - * Defaults to {@link CompressedObjectStrategy#DEFAULT_COMPRESSION_STRATEGY} + * Defaults to {@link CompressionStrategy#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 CompressionStrategy#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} @@ -91,8 +91,8 @@ public class IndexSpec @JsonCreator public IndexSpec( @JsonProperty("bitmap") BitmapSerdeFactory bitmapSerdeFactory, - @JsonProperty("dimensionCompression") CompressedObjectStrategy.CompressionStrategy dimensionCompression, - @JsonProperty("metricCompression") CompressedObjectStrategy.CompressionStrategy metricCompression, + @JsonProperty("dimensionCompression") CompressionStrategy dimensionCompression, + @JsonProperty("metricCompression") CompressionStrategy metricCompression, @JsonProperty("longEncoding") CompressionFactory.LongEncodingStrategy longEncoding ) { @@ -118,13 +118,13 @@ public class IndexSpec } @JsonProperty - public CompressedObjectStrategy.CompressionStrategy getDimensionCompression() + public CompressionStrategy getDimensionCompression() { return dimensionCompression; } @JsonProperty - public CompressedObjectStrategy.CompressionStrategy getMetricCompression() + public CompressionStrategy getMetricCompression() { return metricCompression; } diff --git a/processing/src/main/java/io/druid/segment/LongColumnSelector.java b/processing/src/main/java/io/druid/segment/LongColumnSelector.java index 85ba51d87f6..9f849fe9063 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSelector.java @@ -28,9 +28,6 @@ package io.druid.segment; */ public interface LongColumnSelector extends ColumnValueSelector { - @Override - long getLong(); - /** * @deprecated This method is marked as deprecated in LongColumnSelector to minimize the probability of accidential * calling. "Polymorphism" of LongColumnSelector should be used only when operating on {@link ColumnValueSelector} diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java index b31a5f50454..60dc95cae10 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java @@ -21,9 +21,9 @@ package io.druid.segment; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.IOPeon; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.LongSupplierSerializer; import java.io.IOException; @@ -36,31 +36,31 @@ import java.nio.channels.WritableByteChannel; public class LongColumnSerializer implements GenericColumnSerializer { public static LongColumnSerializer create( - IOPeon ioPeon, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, - CompressedObjectStrategy.CompressionStrategy compression, + CompressionStrategy compression, CompressionFactory.LongEncodingStrategy encoding ) { - return new LongColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression, encoding); + return new LongColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression, encoding); } - private final IOPeon ioPeon; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ByteOrder byteOrder; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionStrategy compression; private final CompressionFactory.LongEncodingStrategy encoding; private LongSupplierSerializer writer; - public LongColumnSerializer( - IOPeon ioPeon, + private LongColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression, + CompressionStrategy compression, CompressionFactory.LongEncodingStrategy encoding ) { - this.ioPeon = ioPeon; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -71,7 +71,7 @@ public class LongColumnSerializer implements GenericColumnSerializer public void open() throws IOException { writer = CompressionFactory.getLongSerializer( - ioPeon, + segmentWriteOutMedium, StringUtils.format("%s.long_column", filenameBase), byteOrder, encoding, @@ -88,21 +88,14 @@ public class LongColumnSerializer implements GenericColumnSerializer } @Override - public void close() throws IOException - { - writer.close(); - } - - @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { return writer.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - writer.writeToChannel(channel, smoosher); + writer.writeTo(channel, smoosher); } - } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java index 5782972520b..19796815169 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java @@ -22,11 +22,10 @@ package io.druid.segment; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.GenericColumn; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.Closeable; -import java.io.File; import java.io.IOException; public class LongDimensionHandler implements DimensionHandler @@ -52,13 +51,16 @@ public class LongDimensionHandler implements DimensionHandler @Override public DimensionMergerV9 makeMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress + IndexSpec indexSpec, + SegmentWriteOutMedium segmentWriteOutMedium, + ColumnCapabilities capabilities, + ProgressIndicator progress ) throws IOException { return new LongDimensionMergerV9( dimensionName, indexSpec, - ioPeon + segmentWriteOutMedium ); } diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java index 4bc45234a5b..ce515ad1020 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java @@ -22,10 +22,10 @@ package io.druid.segment; import com.google.common.base.Throwables; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; -import io.druid.segment.data.IOPeon; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.serde.LongGenericColumnPartSerde; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; import java.nio.IntBuffer; @@ -35,32 +35,30 @@ public class LongDimensionMergerV9 implements DimensionMergerV9 { protected String dimensionName; protected final IndexSpec indexSpec; - protected IOPeon ioPeon; protected LongColumnSerializer serializer; - public LongDimensionMergerV9( + LongDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - IOPeon ioPeon + SegmentWriteOutMedium segmentWriteOutMedium ) { this.dimensionName = dimensionName; this.indexSpec = indexSpec; - this.ioPeon = ioPeon; try { - setupEncodedValueWriter(); + setupEncodedValueWriter(segmentWriteOutMedium); } catch (IOException ioe) { Throwables.propagate(ioe); } } - protected void setupEncodedValueWriter() throws IOException + protected void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); + final CompressionStrategy metCompression = indexSpec.getMetricCompression(); final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); - this.serializer = LongColumnSerializer.create(ioPeon, dimensionName, metCompression, longEncoding); + this.serializer = LongColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression, longEncoding); serializer.open(); } @@ -98,7 +96,6 @@ public class LongDimensionMergerV9 implements DimensionMergerV9 @Override public ColumnDescriptor makeColumnDescriptor() throws IOException { - serializer.close(); final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); builder.setValueType(ValueType.LONG); builder.addSerde( diff --git a/processing/src/main/java/io/druid/segment/MetricHolder.java b/processing/src/main/java/io/druid/segment/MetricHolder.java index 63954a66107..226e6a0c884 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -40,7 +40,6 @@ public class MetricHolder private static final byte[] version = new byte[]{0x0}; private static final SerializerUtils serializerUtils = new SerializerUtils(); - public static MetricHolder fromByteBuffer(ByteBuffer buf, SmooshedFileMapper mapper) throws IOException { return fromByteBuffer(buf, null, mapper); @@ -60,7 +59,7 @@ public class MetricHolder switch (holder.type) { case FLOAT: - holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder(), mapper); + holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); break; case COMPLEX: if (strategy != null) { diff --git a/processing/src/main/java/io/druid/segment/Rowboat.java b/processing/src/main/java/io/druid/segment/Rowboat.java index 1ef2dbf3aa1..c047fd1c567 100644 --- a/processing/src/main/java/io/druid/segment/Rowboat.java +++ b/processing/src/main/java/io/druid/segment/Rowboat.java @@ -51,7 +51,7 @@ public class Rowboat implements Comparable this.rowNum = rowNum; this.handlers = handlers; - this.comprisedRows = new Int2ObjectOpenHashMap<>(); + this.comprisedRows = new Int2ObjectOpenHashMap<>(1); } public long getTimestamp() diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index bfb993afbb4..cd4db8a3db4 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -21,15 +21,14 @@ package io.druid.segment; import com.google.common.primitives.Ints; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.DictionaryEncodedColumn; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import java.io.Closeable; -import java.io.File; import java.lang.reflect.Array; import java.util.Arrays; @@ -199,13 +198,12 @@ public class StringDimensionHandler implements DimensionHandler { private static final Logger log = new Logger(StringDimensionMergerV9.class); - protected static final Indexed EMPTY_STR_DIM_VAL = new ArrayIndexed<>(new String[]{""}, String.class); - protected static final int[] EMPTY_STR_DIM_ARRAY = new int[]{0}; - protected static final Splitter SPLITTER = Splitter.on(","); + private static final Indexed EMPTY_STR_DIM_VAL = new ArrayIndexed<>(new String[]{""}, String.class); + private static final int[] EMPTY_STR_DIM_ARRAY = new int[]{0}; + private static final Splitter SPLITTER = Splitter.on(","); private IndexedIntsWriter encodedValueWriter; - protected String dimensionName; - protected GenericIndexedWriter dictionaryWriter; - protected GenericIndexedWriter bitmapWriter; - protected ByteBufferWriter spatialWriter; - protected ArrayList dimConversions; - protected int cardinality = 0; - protected boolean convertMissingValues = false; - protected boolean hasNull = false; - protected MutableBitmap nullRowsBitmap; - protected IOPeon ioPeon; - protected int rowCount = 0; - protected ColumnCapabilities capabilities; - protected final File outDir; - protected List adapters; - protected ProgressIndicator progress; - protected final IndexSpec indexSpec; - protected IndexMerger.DictionaryMergeIterator dictionaryMergeIterator; + private String dimensionName; + private GenericIndexedWriter dictionaryWriter; + private String firstDictionaryValue; + private int dictionarySize; + private GenericIndexedWriter bitmapWriter; + private ByteBufferWriter spatialWriter; + private ArrayList dimConversions; + private int cardinality = 0; + private boolean convertMissingValues = false; + private boolean hasNull = false; + private MutableBitmap nullRowsBitmap; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private int rowCount = 0; + private ColumnCapabilities capabilities; + private List adapters; + private ProgressIndicator progress; + private final IndexSpec indexSpec; + private IndexMerger.DictionaryMergeIterator dictionaryMergeIterator; public StringDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -106,8 +99,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 this.dimensionName = dimensionName; this.indexSpec = indexSpec; this.capabilities = capabilities; - this.outDir = outDir; - this.ioPeon = ioPeon; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.progress = progress; nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } @@ -158,21 +150,15 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 } String dictFilename = StringUtils.format("%s.dim_values", dimensionName); - dictionaryWriter = new GenericIndexedWriter<>( - ioPeon, - dictFilename, - GenericIndexed.STRING_STRATEGY - ); + dictionaryWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, dictFilename, GenericIndexed.STRING_STRATEGY); + firstDictionaryValue = null; + dictionarySize = 0; dictionaryWriter.open(); cardinality = 0; if (numMergeIndex > 1) { dictionaryMergeIterator = new IndexMerger.DictionaryMergeIterator(dimValueLookups, true); - - while (dictionaryMergeIterator.hasNext()) { - dictionaryWriter.write(dictionaryMergeIterator.next()); - } - + writeDictionary(() -> dictionaryMergeIterator); for (int i = 0; i < adapters.size(); i++) { if (dimValueLookups[i] != null && dictionaryMergeIterator.needConversion(i)) { dimConversions.set(i, dictionaryMergeIterator.conversions[i]); @@ -180,9 +166,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 } cardinality = dictionaryMergeIterator.counter; } else if (numMergeIndex == 1) { - for (String value : dimValueLookup) { - dictionaryWriter.write(value); - } + writeDictionary(dimValueLookup); cardinality = dimValueLookup.size(); } @@ -192,34 +176,49 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 cardinality, System.currentTimeMillis() - dimStartTime ); - dictionaryWriter.close(); setupEncodedValueWriter(); } + private void writeDictionary(Iterable dictionaryValues) throws IOException + { + for (String value : dictionaryValues) { + dictionaryWriter.write(value); + value = Strings.emptyToNull(value); + if (dictionarySize == 0) { + firstDictionaryValue = value; + } + dictionarySize++; + } + } + protected void setupEncodedValueWriter() throws IOException { - final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); + final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); String filenameBase = StringUtils.format("%s.forward_dim", dimensionName); if (capabilities.hasMultipleValues()) { - encodedValueWriter = (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) - ? CompressedVSizeIndexedV3Writer.create( - ioPeon, - filenameBase, - cardinality, - compressionStrategy - ) - : new VSizeIndexedWriter(ioPeon, filenameBase, cardinality); + if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { + encodedValueWriter = CompressedVSizeIndexedV3Writer.create( + segmentWriteOutMedium, + filenameBase, + cardinality, + compressionStrategy + ); + } else { + encodedValueWriter = new VSizeIndexedWriter(segmentWriteOutMedium, cardinality); + } } else { - encodedValueWriter = (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) - ? CompressedVSizeIntsIndexedWriter.create( - ioPeon, - filenameBase, - cardinality, - compressionStrategy - ) - : new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality); + if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { + encodedValueWriter = CompressedVSizeIntsIndexedWriter.create( + segmentWriteOutMedium, + filenameBase, + cardinality, + compressionStrategy + ); + } else { + encodedValueWriter = new VSizeIndexedIntsWriter(segmentWriteOutMedium, cardinality); + } } encodedValueWriter.open(); } @@ -277,87 +276,65 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); String bmpFilename = StringUtils.format("%s.inverted", dimensionName); - bitmapWriter = new GenericIndexedWriter<>(ioPeon, bmpFilename, bitmapSerdeFactory.getObjectStrategy()); + bitmapWriter = new GenericIndexedWriter<>( + segmentWriteOutMedium, + bmpFilename, + indexSpec.getBitmapSerdeFactory().getObjectStrategy() + ); bitmapWriter.open(); bitmapWriter.setObjectsNotSorted(); - // write dim values to one single file because we need to read it - File dimValueFile = IndexIO.makeDimFile(outDir, dimensionName); - try (FileOutputStream fos = new FileOutputStream(dimValueFile)) { - ByteStreams.copy(dictionaryWriter.combineStreams(), fos); + BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory(); + + RTree tree = null; + boolean hasSpatial = capabilities.hasSpatialIndexes(); + if (hasSpatial) { + spatialWriter = new ByteBufferWriter<>( + segmentWriteOutMedium, + new ImmutableRTreeObjectStrategy(bitmapFactory) + ); + spatialWriter.open(); + tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory); } - final MappedByteBuffer dimValsMapped = Files.map(dimValueFile); - try ( - Closeable toCloseEncodedValueWriter = encodedValueWriter; - Closeable toCloseBitmapWriter = bitmapWriter; - // We need to free the ByteBuffers allocated by the dictionary merge iterator here, - // these buffers are used by dictIdSeeker in mergeBitmaps() below. The iterator is created and only used - // in writeMergedValueMetadata(), but the buffers are still used until after mergeBitmaps(). - Closeable toCloseDictionaryMergeIterator = dictionaryMergeIterator; - Closeable dimValsMappedUnmapper = () -> ByteBufferUtils.unmap(dimValsMapped) - ) { - Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); - BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory(); + IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName); - RTree tree = null; - boolean hasSpatial = capabilities.hasSpatialIndexes(); - if (hasSpatial) { - spatialWriter = new ByteBufferWriter<>( - ioPeon, - StringUtils.format("%s.spatial", dimensionName), - new ImmutableRTreeObjectStrategy(bmpFactory) - ); - spatialWriter.open(); - tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory); - } - - IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName); - - //Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result. - for (int dictId = 0; dictId < dimVals.size(); dictId++) { - progress.progress(); - mergeBitmaps( - segmentRowNumConversions, - dimVals, - bmpFactory, - tree, - hasSpatial, - dictIdSeeker, - dictId, - adapters, - dimensionName, - nullRowsBitmap, - bitmapWriter - ); - } - - if (hasSpatial) { - spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); - spatialWriter.close(); - } - - log.info( - "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", - dimensionName, - dimVals.size(), - System.currentTimeMillis() - dimStartTime + //Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result. + for (int dictId = 0; dictId < dictionarySize; dictId++) { + progress.progress(); + mergeBitmaps( + segmentRowNumConversions, + bitmapFactory, + tree, + hasSpatial, + dictIdSeeker, + dictId ); } + + if (hasSpatial) { + spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); + } + + log.info( + "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", + dimensionName, + dictionarySize, + System.currentTimeMillis() - dimStartTime + ); + + if (dictionaryMergeIterator != null) { + dictionaryMergeIterator.close(); + } } - static void mergeBitmaps( + void mergeBitmaps( List segmentRowNumConversions, - Indexed dimVals, BitmapFactory bmpFactory, RTree tree, boolean hasSpatial, IndexSeeker[] dictIdSeeker, - int dictId, - List adapters, - String dimensionName, - MutableBitmap nullRowsBitmap, - GenericIndexedWriter bitmapWriter + int dictId ) throws IOException { List convertedInvertedIndexesToMerge = Lists.newArrayListWithCapacity(adapters.size()); @@ -392,14 +369,14 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 prevRow = row; } - if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) { + if (dictId == 0 && firstDictionaryValue == null) { mergedIndexes.or(nullRowsBitmap); } bitmapWriter.write(bmpFactory.makeImmutableBitmap(mergedIndexes)); if (hasSpatial) { - String dimVal = dimVals.get(dictId); + String dimVal = dictionaryWriter.get(dictId); if (dimVal != null) { List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); float[] coords = new float[stringCoords.size()]; @@ -422,7 +399,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 { // Now write everything boolean hasMultiValue = capabilities.hasMultipleValues(); - final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); + final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); final ColumnDescriptor.Builder builder = ColumnDescriptor.builder(); @@ -434,7 +411,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 .withValue( encodedValueWriter, hasMultiValue, - compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED + compressionStrategy != CompressionStrategy.UNCOMPRESSED ) .withBitmapSerdeFactory(bitmapSerdeFactory) .withBitmapIndex(bitmapWriter) diff --git a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java index 7119012abb8..0793c40f249 100644 --- a/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java +++ b/processing/src/main/java/io/druid/segment/column/ColumnDescriptor.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.segment.serde.ColumnPartSerde; +import io.druid.segment.serde.Serializer; import java.io.IOException; import java.nio.ByteBuffer; @@ -35,7 +36,7 @@ import java.util.List; /** */ -public class ColumnDescriptor +public class ColumnDescriptor implements Serializer { public static Builder builder() { @@ -76,21 +77,21 @@ public class ColumnDescriptor return parts; } - public long numBytes() + @Override + public long getSerializedSize() throws IOException { - long retVal = 0; - + long size = 0; for (ColumnPartSerde part : parts) { - retVal += part.getSerializer().numBytes(); + size += part.getSerializer().getSerializedSize(); } - - return retVal; + return size; } - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { for (ColumnPartSerde part : parts) { - part.getSerializer().write(channel, smoosher); + part.getSerializer().writeTo(channel, smoosher); } } diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java index 6069000176e..287deafe4d5 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java @@ -19,52 +19,49 @@ package io.druid.segment.data; -import com.google.common.io.ByteStreams; -import com.google.common.io.CountingOutputStream; -import com.google.common.primitives.Doubles; -import com.google.common.primitives.Ints; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.writeout.SegmentWriteOutMedium; import java.io.IOException; -import java.io.InputStream; +import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.DoubleBuffer; -import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; public class BlockLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer { - private final IOPeon ioPeon; - private final int sizePer; - private final GenericIndexedWriter> flattener; - private final CompressedObjectStrategy.CompressionStrategy compression; - private final String metaFile; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((BlockLayoutDoubleSupplierSerializer x) -> CompressedDoublesIndexedSupplier.VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> CompressedPools.BUFFER_SIZE / Double.BYTES) + .writeByte(x -> x.compression.getId()); + + private final GenericIndexedWriter flattener; + private final CompressionStrategy compression; - private long metaCount = 0; private int numInserted = 0; - private DoubleBuffer endBuffer; + private ByteBuffer endBuffer; - public BlockLayoutDoubleSupplierSerializer( - IOPeon ioPeon, + BlockLayoutDoubleSupplierSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, - ByteOrder order, - CompressedObjectStrategy.CompressionStrategy compression + ByteOrder byteOrder, + CompressionStrategy compression ) { - this.ioPeon = ioPeon; - this.sizePer = CompressedPools.BUFFER_SIZE / Doubles.BYTES; - this.flattener = new GenericIndexedWriter<>( - ioPeon, filenameBase, CompressedDoubleBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + this.flattener = GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, + filenameBase, + compression, + CompressedPools.BUFFER_SIZE ); - this.metaFile = filenameBase + ".format"; this.compression = compression; - - endBuffer = DoubleBuffer.allocate(sizePer); - endBuffer.mark(); + CompressionStrategy.Compressor compressor = compression.getCompressor(); + Closer closer = segmentWriteOutMedium.getCloser(); + this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder); } @Override @@ -76,48 +73,42 @@ public class BlockLayoutDoubleSupplierSerializer implements DoubleSupplierSerial @Override public void add(double value) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = DoubleBuffer.allocate(sizePer); - endBuffer.mark(); + flattener.write(endBuffer); + endBuffer.clear(); } - endBuffer.put(value); + endBuffer.putDouble(value); ++numInserted; } @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { - return metaCount + flattener.getSerializedSize(); + writeEndBuffer(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - try (InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - flattener.writeToChannel(channel, smoosher); - } + writeEndBuffer(); + metaSerdeHelper.writeTo(channel, this); + flattener.writeTo(channel, smoosher); } - @Override - public void close() throws IOException + private void writeEndBuffer() throws IOException { - 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(CompressedDoublesIndexedSupplier.version); - metaOut.write(Ints.toByteArray(numInserted)); - metaOut.write(Ints.toByteArray(sizePer)); - metaOut.write(compression.getId()); - metaOut.close(); - metaCount = metaOut.getCount(); + if (endBuffer != null) { + endBuffer.flip(); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } + endBuffer = null; } } } diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java index ddb112b5063..d0961c801ab 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java @@ -19,53 +19,48 @@ 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 io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.FloatBuffer; -import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; public class BlockLayoutFloatSupplierSerializer implements FloatSupplierSerializer { - private final IOPeon ioPeon; - private final int sizePer; - private final GenericIndexedWriter> flattener; - private final CompressedObjectStrategy.CompressionStrategy compression; - private final String metaFile; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((BlockLayoutFloatSupplierSerializer x) -> CompressedFloatsIndexedSupplier.VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> CompressedPools.BUFFER_SIZE / Float.BYTES) + .writeByte(x -> x.compression.getId()); + + private final GenericIndexedWriter flattener; + private final CompressionStrategy compression; - private long metaCount = 0; private int numInserted = 0; - private FloatBuffer endBuffer; + private ByteBuffer endBuffer; - public BlockLayoutFloatSupplierSerializer( - IOPeon ioPeon, + BlockLayoutFloatSupplierSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, - ByteOrder order, - CompressedObjectStrategy.CompressionStrategy compression + ByteOrder byteOrder, + CompressionStrategy compression ) { - this.ioPeon = ioPeon; - this.sizePer = CompressedPools.BUFFER_SIZE / Floats.BYTES; - this.flattener = new GenericIndexedWriter<>( - ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer) + this.flattener = GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, + filenameBase, + compression, + CompressedPools.BUFFER_SIZE ); - this.metaFile = filenameBase + ".format"; this.compression = compression; - - endBuffer = FloatBuffer.allocate(sizePer); - endBuffer.mark(); + CompressionStrategy.Compressor compressor = compression.getCompressor(); + Closer closer = segmentWriteOutMedium.getCloser(); + this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder); } @Override @@ -83,59 +78,41 @@ public class BlockLayoutFloatSupplierSerializer implements FloatSupplierSerializ @Override public void add(float value) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = FloatBuffer.allocate(sizePer); - endBuffer.mark(); + flattener.write(endBuffer); + endBuffer.clear(); } - - endBuffer.put(value); + endBuffer.putFloat(value); ++numInserted; } @Override - public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + public long getSerializedSize() throws IOException { - close(); - try (OutputStream out = consolidatedOut.openStream(); - InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(meta, out); - ByteStreams.copy(flattener.combineStreams(), out); - } + writeEndBuffer(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - 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(CompressedFloatsIndexedSupplier.version); - metaOut.write(Ints.toByteArray(numInserted)); - metaOut.write(Ints.toByteArray(sizePer)); - metaOut.write(compression.getId()); - metaOut.close(); - metaCount = metaOut.getCount(); - } + writeEndBuffer(); + metaSerdeHelper.writeTo(channel, this); + flattener.writeTo(channel, smoosher); } - @Override - public long getSerializedSize() + private void writeEndBuffer() throws IOException { - return metaCount + flattener.getSerializedSize(); - } - - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - try (InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - flattener.writeToChannel(channel, smoosher); + if (endBuffer != null) { + endBuffer.flip(); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } + endBuffer = null; } } } diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java index 779665b41a5..85bdebd11f0 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java @@ -20,10 +20,8 @@ package io.druid.segment.data; import com.google.common.base.Supplier; -import com.google.common.primitives.Doubles; import io.druid.collections.ResourceHolder; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -41,18 +39,13 @@ public class BlockLayoutIndexedDoubleSupplier implements Supplier int sizePer, ByteBuffer fromBuffer, ByteOrder order, - CompressedObjectStrategy.CompressionStrategy strategy, - SmooshedFileMapper mapper + CompressionStrategy strategy ) { baseFloatBuffers = GenericIndexed.read( fromBuffer, - VSizeCompressedObjectStrategy.getBufferForOrder( - order, - strategy, - sizePer * Floats.BYTES - ), - mapper + new DecompressingByteBufferObjectStrategy(order, strategy) ); this.totalSize = totalSize; this.sizePer = sizePer; diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java index 7202b5bc02f..54f2ef62d9e 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedLongSupplier.java @@ -23,7 +23,6 @@ import com.google.common.base.Supplier; import io.druid.collections.ResourceHolder; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -43,19 +42,10 @@ public class BlockLayoutIndexedLongSupplier implements Supplier ByteBuffer fromBuffer, ByteOrder order, CompressionFactory.LongEncodingReader reader, - CompressedObjectStrategy.CompressionStrategy strategy, - SmooshedFileMapper fileMapper + CompressionStrategy strategy ) { - baseLongBuffers = GenericIndexed.read( - fromBuffer, - VSizeCompressedObjectStrategy.getBufferForOrder( - order, - strategy, - reader.getNumBytes(sizePer) - ), - fileMapper - ); + baseLongBuffers = GenericIndexed.read(fromBuffer, new DecompressingByteBufferObjectStrategy(order, strategy)); this.totalSize = totalSize; this.sizePer = sizePer; this.baseReader = reader; diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java index abdae5da7a9..f44a91633d8 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java @@ -19,60 +19,50 @@ 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.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; 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 BlockLayoutLongSupplierSerializer implements LongSupplierSerializer { + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((BlockLayoutLongSupplierSerializer x) -> CompressedLongsIndexedSupplier.VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> x.sizePer) + .writeSomething(CompressionFactory.longEncodingWriter(x -> x.writer, x -> x.compression)); - 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 final GenericIndexedWriter flattener; + private final CompressionStrategy compression; private int numInserted = 0; + private int numInsertedForNextFlush; private ByteBuffer endBuffer = null; - public BlockLayoutLongSupplierSerializer( - IOPeon ioPeon, + BlockLayoutLongSupplierSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, - ByteOrder order, + ByteOrder byteOrder, CompressionFactory.LongEncodingWriter writer, - CompressedObjectStrategy.CompressionStrategy compression + 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"; + int bufferSize = writer.getNumBytes(sizePer); + this.flattener = GenericIndexedWriter.ofCompressedByteBuffers(segmentWriteOutMedium, filenameBase, compression, bufferSize); this.writer = writer; this.compression = compression; + CompressionStrategy.Compressor compressor = compression.getCompressor(); + endBuffer = compressor.allocateInBuffer(writer.getNumBytes(sizePer), segmentWriteOutMedium.getCloser()).order(byteOrder); + writer.setBuffer(endBuffer); + numInsertedForNextFlush = sizePer; } @Override @@ -90,15 +80,15 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer @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); + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } + if (numInserted == numInsertedForNextFlush) { + numInsertedForNextFlush += sizePer; + writer.flush(); + endBuffer.flip(); + flattener.write(endBuffer); + endBuffer.clear(); } writer.write(value); @@ -106,50 +96,29 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer } @Override - public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + public long getSerializedSize() throws IOException { - close(); - try (OutputStream out = consolidatedOut.openStream(); - InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(meta, out); - ByteStreams.copy(flattener.combineStreams(), out); - } + writeEndBuffer(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + writeEndBuffer(); + metaSerdeHelper.writeTo(channel, this); + flattener.writeTo(channel, smoosher); + } + + private void writeEndBuffer() 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, FileSmoosher smoosher) throws IOException - { - try (InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - flattener.writeToChannel(channel, smoosher); + endBuffer.flip(); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } + endBuffer = null; } } } diff --git a/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java b/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java index 6ce80212f3a..87ef0d5f863 100644 --- a/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java @@ -23,7 +23,7 @@ import java.nio.ByteBuffer; /** */ -public class ByteBufferSerializer +public class ByteBufferSerializer { public static T read(ByteBuffer buffer, ObjectStrategy strategy) { @@ -34,5 +34,4 @@ public class ByteBufferSerializer return strategy.fromByteBuffer(bufferToUse, size); } - } diff --git a/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java index 49e41bf4870..27c5a08dd41 100644 --- a/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java @@ -19,111 +19,60 @@ package io.druid.segment.data; -import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; -import com.google.common.io.ByteStreams; -import com.google.common.io.CountingOutputStream; -import com.google.common.io.InputSupplier; import com.google.common.primitives.Ints; -import io.druid.common.utils.SerializerUtils; -import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.Serializer; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; -import java.io.Closeable; import java.io.IOException; -import java.io.InputStream; -import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -import java.util.Arrays; /** */ -public class ByteBufferWriter implements Closeable +public class ByteBufferWriter implements Serializer { - private final IOPeon ioPeon; - private final String filenameBase; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final ObjectStrategy strategy; - private CountingOutputStream headerOut = null; - private CountingOutputStream valueOut = null; - private final ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); + private WriteOutBytes headerOut = null; + private WriteOutBytes valueOut = null; - public ByteBufferWriter( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) + public ByteBufferWriter(SegmentWriteOutMedium segmentWriteOutMedium, ObjectStrategy strategy) { - this.ioPeon = ioPeon; - this.filenameBase = filenameBase; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.strategy = strategy; } public void open() throws IOException { - headerOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("header"))); - valueOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("value"))); + headerOut = segmentWriteOutMedium.makeWriteOutBytes(); + valueOut = segmentWriteOutMedium.makeWriteOutBytes(); } public void write(T objectToWrite) throws IOException { - byte[] bytesToWrite = strategy.toBytes(objectToWrite); - SerializerUtils.writeBigEndianIntToOutputStream(headerOut, bytesToWrite.length, helperBuffer); - valueOut.write(bytesToWrite); - } - - private String makeFilename(String suffix) - { - return StringUtils.format("%s.%s", filenameBase, suffix); + long sizeBefore = valueOut.size(); + strategy.writeTo(objectToWrite, valueOut); + headerOut.writeInt(Ints.checkedCast(valueOut.size() - sizeBefore)); } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - headerOut.close(); - valueOut.close(); + return headerOut.size() + valueOut.size(); + } - final long numBytesWritten = headerOut.getCount() + valueOut.getCount(); + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + final long numBytesWritten = headerOut.size() + valueOut.size(); Preconditions.checkState( numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten ); - } - public long getSerializedSize() - { - return headerOut.getCount() + valueOut.getCount(); - } - - public InputSupplier combineStreams() - { - return ByteStreams.join( - Iterables.transform( - Arrays.asList("header", "value"), - new Function>() - { - @Override - public InputSupplier apply(final String input) - { - return new InputSupplier() - { - @Override - public InputStream getInput() throws IOException - { - return ioPeon.makeInputStream(makeFilename(input)); - } - }; - } - } - ) - ); - } - - public void writeToChannel(WritableByteChannel channel) throws IOException - { - try (final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput())) { - ByteStreams.copy(from, channel); - } + headerOut.writeTo(channel); + valueOut.writeTo(channel); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedByteBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedByteBufferObjectStrategy.java deleted file mode 100644 index 96afb159c78..00000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedByteBufferObjectStrategy.java +++ /dev/null @@ -1,70 +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 io.druid.java.util.common.guava.Comparators; - -import java.nio.ByteBuffer; -import java.nio.ByteOrder; - -public class CompressedByteBufferObjectStrategy extends FixedSizeCompressedObjectStrategy -{ - - public static CompressedByteBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer) - { - return new CompressedByteBufferObjectStrategy(order, compression, sizePer); - } - - public CompressedByteBufferObjectStrategy( - ByteOrder order, - CompressionStrategy compression, - final int sizePer - ) - { - super( - order, new BufferConverter() - { - @Override - public ByteBuffer convert(ByteBuffer buf) - { - return buf; - } - - @Override - public int compare(ByteBuffer lhs, ByteBuffer rhs) - { - return Comparators.naturalNullsFirst().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, sizePer - ); - } -} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java deleted file mode 100644 index 5e7778056ba..00000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java +++ /dev/null @@ -1,73 +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.Doubles; -import io.druid.java.util.common.guava.Comparators; - -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.DoubleBuffer; - -public class CompressedDoubleBufferObjectStrategy extends FixedSizeCompressedObjectStrategy -{ - public static CompressedDoubleBufferObjectStrategy getBufferForOrder( - final ByteOrder order, - final CompressionStrategy compression, - final int size - ) - { - return new CompressedDoubleBufferObjectStrategy(order, compression, size); - } - private CompressedDoubleBufferObjectStrategy( - ByteOrder order, - CompressionStrategy compression, - int sizePer - ) - { - super(order, new BufferConverter() - { - @Override - public DoubleBuffer convert(ByteBuffer buf) - { - return buf.asDoubleBuffer(); - } - - @Override - public int compare(DoubleBuffer lhs, DoubleBuffer rhs) - { - return Comparators.naturalNullsFirst().compare(lhs, rhs); - } - - @Override - public int sizeOf(int count) - { - return count * Doubles.BYTES; - } - - @Override - public DoubleBuffer combine(ByteBuffer into, DoubleBuffer from) - { - return into.asDoubleBuffer().put(from); - } - }, compression, sizePer); - } -} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java index 5297f4cb5ba..3023e42a03f 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java @@ -21,7 +21,6 @@ package io.druid.segment.data; import com.google.common.base.Supplier; import io.druid.java.util.common.IAE; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -29,7 +28,7 @@ import java.nio.ByteOrder; public class CompressedDoublesIndexedSupplier { public static final byte LZF_VERSION = 0x1; - public static final byte version = 0x2; + public static final byte VERSION = 0x2; private CompressedDoublesIndexedSupplier() { @@ -37,30 +36,27 @@ public class CompressedDoublesIndexedSupplier public static Supplier fromByteBuffer( ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper mapper + ByteOrder order ) { byte versionFromBuffer = buffer.get(); - if (versionFromBuffer == LZF_VERSION || versionFromBuffer == version) { + if (versionFromBuffer == LZF_VERSION || versionFromBuffer == VERSION) { final int totalSize = buffer.getInt(); final int sizePer = buffer.getInt(); - CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; - if (versionFromBuffer == version) { + CompressionStrategy compression = CompressionStrategy.LZF; + if (versionFromBuffer == VERSION) { byte compressionId = buffer.get(); - compression = CompressedObjectStrategy.CompressionStrategy.forId(compressionId); + compression = CompressionStrategy.forId(compressionId); } return CompressionFactory.getDoubleSupplier( totalSize, sizePer, buffer.asReadOnlyBuffer(), order, - compression, - mapper + compression ); } throw new IAE("Unknown version[%s]", versionFromBuffer); } - } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java deleted file mode 100644 index 09bb6b5819d..00000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java +++ /dev/null @@ -1,72 +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.Floats; -import io.druid.java.util.common.guava.Comparators; - -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.FloatBuffer; - -/** -*/ -public class CompressedFloatBufferObjectStrategy extends FixedSizeCompressedObjectStrategy -{ - public static CompressedFloatBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer) - { - return new CompressedFloatBufferObjectStrategy(order, compression, sizePer); - } - - private CompressedFloatBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) - { - super( - order, - new BufferConverter() - { - @Override - public FloatBuffer convert(ByteBuffer buf) - { - return buf.asFloatBuffer(); - } - - @Override - public int compare(FloatBuffer lhs, FloatBuffer rhs) - { - return Comparators.naturalNullsFirst().compare(lhs, rhs); - } - - @Override - public int sizeOf(int count) - { - return count * Floats.BYTES; - } - - @Override - public FloatBuffer combine(ByteBuffer into, FloatBuffer from) - { - return into.asFloatBuffer().put(from); - } - }, - compression, - sizePer - ); - } -} 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 8fb80677a9a..db0477d6653 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -20,32 +20,40 @@ package io.druid.segment.data; import com.google.common.base.Supplier; -import com.google.common.primitives.Ints; +import io.druid.io.Channels; import io.druid.java.util.common.IAE; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.serde.Serializer; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; -public class CompressedFloatsIndexedSupplier implements Supplier +public class CompressedFloatsIndexedSupplier implements Supplier, Serializer { public static final byte LZF_VERSION = 0x1; - public static final byte version = 0x2; + public static final byte VERSION = 0x2; + + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((CompressedFloatsIndexedSupplier x) -> VERSION) + .writeInt(x -> x.totalSize) + .writeInt(x -> x.sizePer) + .writeByte(x -> x.compression.getId()); private final int totalSize; private final int sizePer; private final ByteBuffer buffer; private final Supplier supplier; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionStrategy compression; CompressedFloatsIndexedSupplier( int totalSize, int sizePer, ByteBuffer buffer, Supplier supplier, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { this.totalSize = totalSize; @@ -61,43 +69,37 @@ public class CompressedFloatsIndexedSupplier implements Supplier return supplier.get(); } - public long getSerializedSize() + @Override + public long getSerializedSize() throws IOException { - return buffer.remaining() + 1 + 4 + 4 + 1; + return metaSerdeHelper.size(this) + (long) buffer.remaining(); } - public void writeToChannel(WritableByteChannel channel) throws IOException + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) 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()})); - channel.write(buffer.asReadOnlyBuffer()); + metaSerdeHelper.writeTo(channel, this); + Channels.writeFully(channel, buffer.asReadOnlyBuffer()); } - public static CompressedFloatsIndexedSupplier fromByteBuffer( - ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper mapper - ) + public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); - if (versionFromBuffer == LZF_VERSION || versionFromBuffer == version) { + if (versionFromBuffer == LZF_VERSION || versionFromBuffer == VERSION) { final int totalSize = buffer.getInt(); final int sizePer = buffer.getInt(); - CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; - if (versionFromBuffer == version) { + CompressionStrategy compression = CompressionStrategy.LZF; + if (versionFromBuffer == VERSION) { byte compressionId = buffer.get(); - compression = CompressedObjectStrategy.CompressionStrategy.forId(compressionId); + compression = CompressionStrategy.forId(compressionId); } Supplier supplier = CompressionFactory.getFloatSupplier( totalSize, sizePer, buffer.asReadOnlyBuffer(), order, - compression, - mapper + compression ); return new CompressedFloatsIndexedSupplier( totalSize, diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedIntBufferObjectStrategy.java deleted file mode 100644 index b3fd010d0ee..00000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntBufferObjectStrategy.java +++ /dev/null @@ -1,71 +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.Ints; -import io.druid.java.util.common.guava.Comparators; - -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.IntBuffer; - -public class CompressedIntBufferObjectStrategy extends FixedSizeCompressedObjectStrategy -{ - - public static CompressedIntBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer) - { - return new CompressedIntBufferObjectStrategy(order, compression, sizePer); - } - - private CompressedIntBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) - { - super( - order, - new BufferConverter() - { - @Override - public IntBuffer convert(ByteBuffer buf) - { - return buf.asIntBuffer(); - } - - @Override - public int compare(IntBuffer lhs, IntBuffer rhs) - { - return Comparators.naturalNullsFirst().compare(lhs, rhs); - } - - @Override - public int sizeOf(int count) - { - return count * Ints.BYTES; - } - - @Override - public IntBuffer combine(ByteBuffer into, IntBuffer from) - { - return into.asIntBuffer().put(from); - } - }, - compression, - sizePer - ); - } -} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java index b4ead1f97dd..888a0bcdd71 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java @@ -19,16 +19,19 @@ package io.druid.segment.data; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; import io.druid.java.util.common.IAE; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; +import it.unimi.dsi.fastutil.ints.IntArrayList; import java.io.IOException; import java.nio.ByteBuffer; @@ -36,24 +39,28 @@ import java.nio.ByteOrder; import java.nio.IntBuffer; import java.nio.channels.WritableByteChannel; import java.util.Iterator; -import java.util.List; public class CompressedIntsIndexedSupplier implements WritableSupplier { public static final byte VERSION = 0x2; public static final int MAX_INTS_IN_BUFFER = CompressedPools.BUFFER_SIZE / Ints.BYTES; + private static MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((CompressedIntsIndexedSupplier x) -> VERSION) + .writeInt(x -> x.totalSize) + .writeInt(x -> x.sizePer) + .writeByte(x -> x.compression.getId()); private final int totalSize; private final int sizePer; - private final GenericIndexed> baseIntBuffers; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final GenericIndexed> baseIntBuffers; + private final CompressionStrategy compression; - CompressedIntsIndexedSupplier( + private CompressedIntsIndexedSupplier( int totalSize, int sizePer, - GenericIndexed> baseIntBuffers, - CompressedObjectStrategy.CompressionStrategy compression + GenericIndexed> baseIntBuffers, + CompressionStrategy compression ) { this.totalSize = totalSize; @@ -91,55 +98,36 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier> getBaseIntBuffers() + @VisibleForTesting + GenericIndexed getBaseIntBuffers() { return baseIntBuffers; } - public static CompressedIntsIndexedSupplier fromByteBuffer( - ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper fileMapper - ) + public static CompressedIntsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); if (versionFromBuffer == VERSION) { final int totalSize = buffer.getInt(); final int sizePer = buffer.getInt(); - final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId( - buffer.get() - ); + final CompressionStrategy compression = CompressionStrategy.forId(buffer.get()); return new CompressedIntsIndexedSupplier( totalSize, sizePer, - GenericIndexed.read( - buffer, - CompressedIntBufferObjectStrategy.getBufferForOrder(order, compression, sizePer), - fileMapper - ), + GenericIndexed.read(buffer, new DecompressingByteBufferObjectStrategy(order, compression)), compression ); } @@ -147,8 +135,13 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier>() + GenericIndexed.ofCompressedByteBuffers( + new Iterable() { @Override - public Iterator> iterator() + public Iterator iterator() { - return new Iterator>() + return new Iterator() { - IntBuffer myBuffer = buffer.asReadOnlyBuffer(); + final IntBuffer myBuffer = buffer.asReadOnlyBuffer(); + final ByteBuffer retVal = compression + .getCompressor() + .allocateInBuffer(chunkFactor * Ints.BYTES, closer) + .order(byteOrder); + final IntBuffer retValAsIntBuffer = retVal.asIntBuffer(); @Override public boolean hasNext() @@ -175,16 +173,17 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier next() + public ByteBuffer next() { - IntBuffer retVal = myBuffer.asReadOnlyBuffer(); - + int initialLimit = myBuffer.limit(); if (chunkFactor < myBuffer.remaining()) { - retVal.limit(retVal.position() + chunkFactor); + myBuffer.limit(myBuffer.position() + chunkFactor); } - myBuffer.position(myBuffer.position() + retVal.remaining()); - - return StupidResourceHolder.create(retVal); + retValAsIntBuffer.clear(); + retValAsIntBuffer.put(myBuffer); + myBuffer.limit(initialLimit); + retVal.clear().limit(retValAsIntBuffer.position() * Ints.BYTES); + return retVal; } @Override @@ -195,17 +194,22 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier list, + final IntArrayList list, final int chunkFactor, final ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression, + final Closer closer ) { Preconditions.checkArgument( @@ -215,14 +219,18 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier>() + GenericIndexed.ofCompressedByteBuffers( + new Iterable() { @Override - public Iterator> iterator() + public Iterator iterator() { - return new Iterator>() + return new Iterator() { + private final ByteBuffer retVal = compression + .getCompressor() + .allocateInBuffer(chunkFactor * Ints.BYTES, closer) + .order(byteOrder); int position = 0; @Override @@ -232,21 +240,15 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier next() + public ByteBuffer next() { - IntBuffer retVal = IntBuffer.allocate(chunkFactor); - - if (chunkFactor > list.size() - position) { - retVal.limit(list.size() - position); + int blockSize = Math.min(list.size() - position, chunkFactor); + retVal.clear(); + for (int limit = position + blockSize; position < limit; position++) { + retVal.putInt(list.getInt(position)); } - final List ints = list.subList(position, position + retVal.remaining()); - for (int value : ints) { - retVal.put(value); - } - retVal.rewind(); - position += retVal.remaining(); - - return StupidResourceHolder.create(retVal); + retVal.flip(); + return retVal; } @Override @@ -257,7 +259,10 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier> singleThreadedIntBuffers = baseIntBuffers.singleThreaded(); + final Indexed> singleThreadedIntBuffers = baseIntBuffers.singleThreaded(); int currIndex = -1; - ResourceHolder holder; + ResourceHolder holder; IntBuffer buffer; @Override @@ -294,7 +299,7 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((CompressedIntsIndexedWriter x) -> VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> x.chunkFactor) + .writeByte(x -> x.compression.getId()); + private final int chunkFactor; - private final CompressedObjectStrategy.CompressionStrategy compression; - private final GenericIndexedWriter> flattener; - private IntBuffer endBuffer; + private final CompressionStrategy compression; + private final GenericIndexedWriter flattener; + private ByteBuffer endBuffer; private int numInserted; - public CompressedIntsIndexedWriter( - final IOPeon ioPeon, + CompressedIntsIndexedWriter( + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final int chunkFactor, final ByteOrder byteOrder, - final CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression ) { - this(chunkFactor, compression, new GenericIndexedWriter<>( - ioPeon, filenameBase, CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) - )); + this( + segmentWriteOutMedium, + chunkFactor, + byteOrder, + compression, + GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, + filenameBase, + compression, + chunkFactor * Integer.BYTES + ) + ); } - public CompressedIntsIndexedWriter( + CompressedIntsIndexedWriter( + final SegmentWriteOutMedium segmentWriteOutMedium, final int chunkFactor, - final CompressedObjectStrategy.CompressionStrategy compression, - GenericIndexedWriter> flattener + final ByteOrder byteOrder, + final CompressionStrategy compression, + final GenericIndexedWriter flattener ) { this.chunkFactor = chunkFactor; this.compression = compression; - this.endBuffer = IntBuffer.allocate(chunkFactor); - this.numInserted = 0; this.flattener = flattener; + CompressionStrategy.Compressor compressor = compression.getCompressor(); + Closer closer = segmentWriteOutMedium.getCloser(); + this.endBuffer = compressor.allocateInBuffer(chunkFactor * Integer.BYTES, closer).order(byteOrder); + this.numInserted = 0; } @Override @@ -78,48 +96,41 @@ public class CompressedIntsIndexedWriter extends SingleValueIndexedIntsWriter @Override protected void addValue(int val) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = IntBuffer.allocate(chunkFactor); + flattener.write(endBuffer); + endBuffer.clear(); } - endBuffer.put(val); + endBuffer.putInt(val); numInserted++; } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - try { - if (numInserted > 0) { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); + writeEndBuffer(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + writeEndBuffer(); + metaSerdeHelper.writeTo(channel, this); + flattener.writeTo(channel, smoosher); + } + + private void writeEndBuffer() throws IOException + { + if (endBuffer != null) { + endBuffer.flip(); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); } endBuffer = null; } - finally { - flattener.close(); - } - } - - @Override - public long getSerializedSize() - { - return 1 + // version - Ints.BYTES + // numInserted - Ints.BYTES + // chunkFactor - 1 + // compression id - flattener.getSerializedSize(); - } - - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - channel.write(ByteBuffer.wrap(new byte[]{VERSION})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - flattener.writeToChannel(channel, smoosher); } } 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 8d9edcdb9ff..b408b54de6a 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -20,9 +20,11 @@ package io.druid.segment.data; import com.google.common.base.Supplier; -import com.google.common.primitives.Ints; +import io.druid.io.Channels; import io.druid.java.util.common.IAE; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.serde.Serializer; import java.io.IOException; import java.nio.ByteBuffer; @@ -31,17 +33,32 @@ import java.nio.channels.WritableByteChannel; /** */ -public class CompressedLongsIndexedSupplier implements Supplier +public class CompressedLongsIndexedSupplier implements Supplier, Serializer { public static final byte LZF_VERSION = 0x1; - public static final byte version = 0x2; + public static final byte VERSION = 0x2; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((CompressedLongsIndexedSupplier x) -> VERSION) + .writeInt(x -> x.totalSize) + .writeInt(x -> x.sizePer) + .maybeWriteByte( + x -> x.encoding != CompressionFactory.LEGACY_LONG_ENCODING_FORMAT, + x -> CompressionFactory.setEncodingFlag(x.compression.getId()) + ) + .writeByte(x -> { + if (x.encoding != CompressionFactory.LEGACY_LONG_ENCODING_FORMAT) { + return x.encoding.getId(); + } else { + return x.compression.getId(); + } + }); private final int totalSize; private final int sizePer; private final ByteBuffer buffer; private final Supplier supplier; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionStrategy compression; private final CompressionFactory.LongEncodingFormat encoding; CompressedLongsIndexedSupplier( @@ -49,7 +66,7 @@ public class CompressedLongsIndexedSupplier implements Supplier int sizePer, ByteBuffer buffer, Supplier supplier, - CompressedObjectStrategy.CompressionStrategy compression, + CompressionStrategy compression, CompressionFactory.LongEncodingFormat encoding ) { @@ -67,45 +84,35 @@ public class CompressedLongsIndexedSupplier implements Supplier return supplier.get(); } - public long getSerializedSize() + @Override + public long getSerializedSize() throws IOException { - return buffer.remaining() + 1 + 4 + 4 + 1 + (encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT ? 0 : 1); + return metaSerdeHelper.size(this) + (long) buffer.remaining(); } - public void writeToChannel(WritableByteChannel channel) throws IOException + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{version})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); - 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()); + metaSerdeHelper.writeTo(channel, this); + Channels.writeFully(channel, buffer.asReadOnlyBuffer()); } - public static CompressedLongsIndexedSupplier fromByteBuffer( - ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper fileMapper - ) + public static CompressedLongsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); - if (versionFromBuffer == LZF_VERSION || versionFromBuffer == version) { + if (versionFromBuffer == LZF_VERSION || versionFromBuffer == VERSION) { final int totalSize = buffer.getInt(); final int sizePer = buffer.getInt(); - CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.LZF; + CompressionStrategy compression = CompressionStrategy.LZF; CompressionFactory.LongEncodingFormat encoding = CompressionFactory.LEGACY_LONG_ENCODING_FORMAT; - if (versionFromBuffer == version) { + 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); + compression = CompressionStrategy.forId(compressionId); } Supplier supplier = CompressionFactory.getLongSupplier( totalSize, @@ -113,8 +120,7 @@ public class CompressedLongsIndexedSupplier implements Supplier buffer.asReadOnlyBuffer(), order, encoding, - compression, - fileMapper + compression ); return new CompressedLongsIndexedSupplier( totalSize, diff --git a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java deleted file mode 100644 index 92b6736927f..00000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ /dev/null @@ -1,402 +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.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonValue; -import com.google.common.collect.Maps; -import com.ning.compress.BufferRecycler; -import com.ning.compress.lzf.LZFDecoder; -import com.ning.compress.lzf.LZFEncoder; -import io.druid.collections.ResourceHolder; -import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.logger.Logger; -import io.druid.segment.CompressedPools; -import net.jpountz.lz4.LZ4Factory; -import net.jpountz.lz4.LZ4SafeDecompressor; -import org.apache.commons.lang.ArrayUtils; - -import java.io.IOException; -import java.nio.Buffer; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Map; - -/** - */ -public class CompressedObjectStrategy implements ObjectStrategy> -{ - 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 enum CompressionStrategy - { - LZF((byte) 0x0) { - @Override - public Decompressor getDecompressor() - { - return LZFDecompressor.defaultDecompressor; - } - - @Override - public Compressor getCompressor() - { - return LZFCompressor.defaultCompressor; - } - }, - - LZ4((byte) 0x1) { - @Override - public Decompressor getDecompressor() - { - return LZ4Decompressor.defaultDecompressor; - } - - @Override - public Compressor getCompressor() - { - return LZ4Compressor.defaultCompressor; - } - }, - UNCOMPRESSED((byte) 0xFF) { - @Override - public Decompressor getDecompressor() - { - return UncompressedDecompressor.defaultDecompressor; - } - - @Override - 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; - - CompressionStrategy(byte id) - { - this.id = id; - } - - public byte getId() - { - return id; - } - - public abstract Compressor getCompressor(); - - public abstract Decompressor getDecompressor(); - - @JsonValue - @Override - public String toString() - { - return StringUtils.toLowerCase(this.name()); - } - - @JsonCreator - public static CompressionStrategy fromString(String name) - { - return valueOf(StringUtils.toUpperCase(name)); - } - - static final Map idMap = Maps.newHashMap(); - - static { - for (CompressionStrategy strategy : CompressionStrategy.values()) { - idMap.put(strategy.getId(), strategy); - } - } - - 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 interface Decompressor - { - /** - * Implementations of this method are expected to call out.flip() after writing to the output buffer - * - * @param in - * @param numBytes - * @param out - */ - void decompress(ByteBuffer in, int numBytes, ByteBuffer out); - - void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize); - } - - public interface Compressor - { - /** - * Currently assumes buf is an array backed ByteBuffer - * - * @param bytes - * - * @return - */ - byte[] compress(byte[] bytes); - } - - public static class UncompressedCompressor implements Compressor - { - private static final UncompressedCompressor defaultCompressor = new UncompressedCompressor(); - - @Override - public byte[] compress(byte[] bytes) - { - return bytes; - } - } - - public static class UncompressedDecompressor implements Decompressor - { - private static final UncompressedDecompressor defaultDecompressor = new UncompressedDecompressor(); - - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) - { - final ByteBuffer copyBuffer = in.duplicate(); - copyBuffer.limit(copyBuffer.position() + numBytes); - out.put(copyBuffer).flip(); - in.position(in.position() + numBytes); - } - - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize) - { - decompress(in, numBytes, out); - } - } - - public static class LZFDecompressor implements Decompressor - { - private static final LZFDecompressor defaultDecompressor = new LZFDecompressor(); - - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) - { - final byte[] bytes = new byte[numBytes]; - in.get(bytes); - - try (final ResourceHolder outputBytesHolder = CompressedPools.getOutputBytes()) { - final byte[] outputBytes = outputBytesHolder.get(); - final int numDecompressedBytes = LZFDecoder.decode(bytes, outputBytes); - out.put(outputBytes, 0, numDecompressedBytes); - out.flip(); - } - catch (IOException e) { - log.error(e, "Error decompressing data"); - } - } - - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize) - { - decompress(in, numBytes, out); - } - } - - public static class LZFCompressor implements Compressor - { - private static final LZFCompressor defaultCompressor = new LZFCompressor(); - - @Override - public byte[] compress(byte[] bytes) - { - try (final ResourceHolder bufferRecycler = CompressedPools.getBufferRecycler()) { - return LZFEncoder.encode(bytes, 0, bytes.length, bufferRecycler.get()); - } - } - } - - public static class LZ4Decompressor implements Decompressor - { - private static final LZ4SafeDecompressor lz4Safe = LZ4Factory.fastestInstance().safeDecompressor(); - private static final LZ4Decompressor defaultDecompressor = new LZ4Decompressor(); - - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) - { - // Since decompressed size is NOT known, must use lz4Safe - // lz4Safe.decompress does not modify buffer positions - final int numDecompressedBytes = lz4Safe.decompress( - in, - in.position(), - numBytes, - out, - out.position(), - out.remaining() - ); - out.limit(out.position() + numDecompressedBytes); - } - - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize) - { - // lz4Safe.decompress does not modify buffer positions. - // Using lz4Safe API for forward-compatibility with https://github.com/druid-io/druid/pull/4762, which doesn't - // always compressed blocks of the same size. - lz4Safe.decompress( - in, - in.position(), - numBytes, - out, - out.position(), - decompressedSize - ); - out.limit(out.position() + decompressedSize); - } - } - - public static class LZ4Compressor implements Compressor - { - private static final LZ4Compressor defaultCompressor = new LZ4Compressor(); - private static final net.jpountz.lz4.LZ4Compressor lz4High = LZ4Factory.fastestInstance().highCompressor(); - - @Override - public byte[] compress(byte[] bytes) - { - return lz4High.compress(bytes); - } - } - - protected final ByteOrder order; - protected final BufferConverter converter; - protected final Decompressor decompressor; - private final Compressor compressor; - - protected CompressedObjectStrategy( - final ByteOrder order, - final BufferConverter converter, - final CompressionStrategy compression - ) - { - this.order = order; - this.converter = converter; - this.decompressor = compression.getDecompressor(); - this.compressor = compression.getCompressor(); - } - - @Override - @SuppressWarnings("unchecked") - public Class> getClazz() - { - return (Class) ResourceHolder.class; - } - - @Override - public ResourceHolder fromByteBuffer(ByteBuffer buffer, int numBytes) - { - final ResourceHolder bufHolder = CompressedPools.getByteBuf(order); - final ByteBuffer buf = bufHolder.get(); - buf.position(0); - buf.limit(buf.capacity()); - - decompress(buffer, numBytes, buf); - return new ResourceHolder() - { - @Override - public T get() - { - return converter.convert(buf); - } - - @Override - public void close() - { - bufHolder.close(); - } - }; - } - - protected void decompress( - ByteBuffer buffer, - int numBytes, - ByteBuffer buf - ) - { - decompressor.decompress(buffer, numBytes, buf); - } - - @Override - public byte[] toBytes(ResourceHolder holder) - { - T val = holder.get(); - ByteBuffer buf = bufferFor(val); - converter.combine(buf, val); - return compressor.compress(buf.array()); - } - - protected ByteBuffer bufferFor(T val) - { - return ByteBuffer.allocate(converter.sizeOf(val.remaining())).order(order); - } - - @Override - public int compare(ResourceHolder o1, ResourceHolder o2) - { - return converter.compare(o1.get(), o2.get()); - } - - public interface BufferConverter - { - T convert(ByteBuffer buf); - - int compare(T lhs, T rhs); - - int sizeOf(int count); - - T combine(ByteBuffer into, T from); - } -} diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java index 660127cdeb1..b5b2574f7fa 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java @@ -17,46 +17,46 @@ * under the License. */ -/** - * Streams array of integers out in the binary format described by CompressedVSizeIndexedV3Supplier - */ + package io.druid.segment.data; -import io.druid.java.util.common.StringUtils; +import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedVSizeIndexedV3Supplier; import io.druid.segment.IndexIO; +import it.unimi.dsi.fastutil.ints.IntList; +import it.unimi.dsi.fastutil.ints.IntLists; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import java.util.ArrayList; -import java.util.List; +/** + * Streams array of integers out in the binary format described by CompressedVSizeIndexedV3Supplier + */ public class CompressedVSizeIndexedV3Writer extends MultiValueIndexedIntsWriter { private static final byte VERSION = CompressedVSizeIndexedV3Supplier.VERSION; - private static final List EMPTY_LIST = new ArrayList<>(); - public static CompressedVSizeIndexedV3Writer create( - final IOPeon ioPeon, + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final int maxValue, - final CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression ) { return new CompressedVSizeIndexedV3Writer( new CompressedIntsIndexedWriter( - ioPeon, - StringUtils.format("%s.offsets", filenameBase), + segmentWriteOutMedium, + filenameBase, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, IndexIO.BYTE_ORDER, compression ), new CompressedVSizeIntsIndexedWriter( - ioPeon, - StringUtils.format("%s.values", filenameBase), + segmentWriteOutMedium, + filenameBase, maxValue, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), IndexIO.BYTE_ORDER, @@ -68,8 +68,9 @@ public class CompressedVSizeIndexedV3Writer extends MultiValueIndexedIntsWriter private final CompressedIntsIndexedWriter offsetWriter; private final CompressedVSizeIntsIndexedWriter valueWriter; private int offset; + private boolean lastOffsetWritten = false; - public CompressedVSizeIndexedV3Writer( + CompressedVSizeIndexedV3Writer( CompressedIntsIndexedWriter offsetWriter, CompressedVSizeIntsIndexedWriter valueWriter ) @@ -87,43 +88,42 @@ public class CompressedVSizeIndexedV3Writer extends MultiValueIndexedIntsWriter } @Override - protected void addValues(List vals) throws IOException + protected void addValues(IntList vals) throws IOException { + if (lastOffsetWritten) { + throw new IllegalStateException("written out already"); + } if (vals == null) { - vals = EMPTY_LIST; + vals = IntLists.EMPTY_LIST; } offsetWriter.add(offset); - for (Integer val : vals) { - valueWriter.add(val); + for (int i = 0; i < vals.size(); i++) { + valueWriter.add(vals.getInt(i)); } offset += vals.size(); } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - try { + writeLastOffset(); + return 1 + offsetWriter.getSerializedSize() + valueWriter.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + writeLastOffset(); + Channels.writeFully(channel, ByteBuffer.wrap(new byte[]{VERSION})); + offsetWriter.writeTo(channel, smoosher); + valueWriter.writeTo(channel, smoosher); + } + + private void writeLastOffset() throws IOException + { + if (!lastOffsetWritten) { offsetWriter.add(offset); + lastOffsetWritten = true; } - finally { - offsetWriter.close(); - valueWriter.close(); - } - } - - @Override - public long getSerializedSize() - { - return 1 + // version - offsetWriter.getSerializedSize() + - valueWriter.getSerializedSize(); - } - - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - channel.write(ByteBuffer.wrap(new byte[]{VERSION})); - offsetWriter.writeToChannel(channel, smoosher); - valueWriter.writeToChannel(channel, smoosher); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java index 41b407eeabe..62535137428 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java @@ -25,12 +25,15 @@ import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import com.google.common.primitives.Shorts; import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; +import io.druid.common.utils.ByteUtils; import io.druid.java.util.common.IAE; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; +import it.unimi.dsi.fastutil.ints.IntList; import java.io.IOException; import java.nio.ByteBuffer; @@ -39,26 +42,32 @@ import java.nio.IntBuffer; import java.nio.ShortBuffer; import java.nio.channels.WritableByteChannel; import java.util.Iterator; -import java.util.List; public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier { public static final byte VERSION = 0x2; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((CompressedVSizeIntsIndexedSupplier x) -> VERSION) + .writeByte(x -> ByteUtils.checkedCast(x.numBytes)) + .writeInt(x -> x.totalSize) + .writeInt(x -> x.sizePer) + .writeByte(x -> x.compression.getId()); + private final int totalSize; private final int sizePer; private final int numBytes; private final int bigEndianShift; private final int littleEndianMask; private final GenericIndexed> baseBuffers; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionStrategy compression; - CompressedVSizeIntsIndexedSupplier( + private CompressedVSizeIntsIndexedSupplier( int totalSize, int sizePer, int numBytes, GenericIndexed> baseBuffers, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { Preconditions.checkArgument( @@ -79,10 +88,10 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier list, + final IntList list, final int maxValue, final int chunkFactor, final ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression, + final Closer closer ) { final int numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue); - final int chunkBytes = chunkFactor * numBytes + bufferPadding(numBytes); + final int chunkBytes = chunkFactor * numBytes; Preconditions.checkArgument( chunkFactor <= maxIntsInBufferForBytes(numBytes), @@ -201,15 +195,19 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier>() + GenericIndexed.ofCompressedByteBuffers( + new Iterable() { @Override - public Iterator> iterator() + public Iterator iterator() { - return new Iterator>() + return new Iterator() { int position = 0; + private final ByteBuffer retVal = + compression.getCompressor().allocateInBuffer(chunkBytes, closer).order(byteOrder); + private final boolean isBigEndian = byteOrder.equals(ByteOrder.BIG_ENDIAN); + private final ByteBuffer helperBuf = ByteBuffer.allocate(Ints.BYTES).order(byteOrder); @Override public boolean hasNext() @@ -218,35 +216,27 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier next() + public ByteBuffer next() { - ByteBuffer retVal = ByteBuffer - .allocate(chunkBytes) - .order(byteOrder); + retVal.clear(); + int elementCount = Math.min(list.size() - position, chunkFactor); + retVal.limit(numBytes * elementCount); - if (chunkFactor > list.size() - position) { - retVal.limit((list.size() - position) * numBytes); - } else { - retVal.limit(chunkFactor * numBytes); - } - - final List ints = list.subList(position, position + retVal.remaining() / numBytes); - final ByteBuffer buf = ByteBuffer - .allocate(Ints.BYTES) - .order(byteOrder); - final boolean bigEndian = byteOrder.equals(ByteOrder.BIG_ENDIAN); - for (int value : ints) { - buf.putInt(0, value); - if (bigEndian) { - retVal.put(buf.array(), Ints.BYTES - numBytes, numBytes); - } else { - retVal.put(buf.array(), 0, numBytes); - } + for (int limit = position + elementCount; position < limit; position++) { + writeIntToRetVal(list.getInt(position)); } retVal.rewind(); - position += retVal.remaining() / numBytes; + return retVal; + } - return StupidResourceHolder.create(retVal); + private void writeIntToRetVal(int value) + { + helperBuf.putInt(0, value); + if (isBigEndian) { + retVal.put(helperBuf.array(), Ints.BYTES - numBytes, numBytes); + } else { + retVal.put(helperBuf.array(), 0, numBytes); + } } @Override @@ -257,7 +247,10 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((CompressedVSizeIntsIndexedWriter x) -> VERSION) + .writeByte(x -> ByteUtils.checkedCast(x.numBytes)) + .writeInt(x -> x.numInserted) + .writeInt(x -> x.chunkFactor) + .writeByte(x -> x.compression.getId()); + + public static CompressedVSizeIntsIndexedWriter create( + final SegmentWriteOutMedium segmentWriteOutMedium, + final String filenameBase, + final int maxValue, + final CompressionStrategy compression + ) + { + return new CompressedVSizeIntsIndexedWriter( + segmentWriteOutMedium, + filenameBase, + maxValue, + CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), + IndexIO.BYTE_ORDER, + compression + ); + } + private final int numBytes; private final int chunkFactor; - private final int chunkBytes; - private final ByteOrder byteOrder; - private final CompressedObjectStrategy.CompressionStrategy compression; - private final GenericIndexedWriter> flattener; + private final boolean isBigEndian; + private final CompressionStrategy compression; + private final GenericIndexedWriter flattener; private final ByteBuffer intBuffer; private ByteBuffer endBuffer; private int numInserted; - public CompressedVSizeIntsIndexedWriter( - final IOPeon ioPeon, + CompressedVSizeIntsIndexedWriter( + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final int maxValue, final int chunkFactor, final ByteOrder byteOrder, - final CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression ) { this( + segmentWriteOutMedium, maxValue, chunkFactor, byteOrder, compression, - new GenericIndexedWriter<>( - ioPeon, + GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, filenameBase, - CompressedByteBufferObjectStrategy.getBufferForOrder( - byteOrder, - compression, - sizePer(maxValue, chunkFactor) - ) + compression, + sizePer(maxValue, chunkFactor) ) ); } - public CompressedVSizeIntsIndexedWriter( + CompressedVSizeIntsIndexedWriter( + final SegmentWriteOutMedium segmentWriteOutMedium, final int maxValue, final int chunkFactor, final ByteOrder byteOrder, - final CompressedObjectStrategy.CompressionStrategy compression, - final GenericIndexedWriter writer + final CompressionStrategy compression, + final GenericIndexedWriter flattener ) { this.numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue); this.chunkFactor = chunkFactor; - this.chunkBytes = chunkFactor * numBytes + CompressedVSizeIntsIndexedSupplier.bufferPadding(numBytes); - this.byteOrder = byteOrder; + int chunkBytes = chunkFactor * numBytes; + this.isBigEndian = byteOrder.equals(ByteOrder.BIG_ENDIAN); this.compression = compression; - this.flattener = writer; + this.flattener = flattener; this.intBuffer = ByteBuffer.allocate(Ints.BYTES).order(byteOrder); - this.endBuffer = ByteBuffer.allocate(chunkBytes).order(byteOrder); - this.endBuffer.limit(numBytes * chunkFactor); + CompressionStrategy.Compressor compressor = compression.getCompressor(); + this.endBuffer = compressor.allocateInBuffer(chunkBytes, segmentWriteOutMedium.getCloser()).order(byteOrder); this.numInserted = 0; } - public static CompressedVSizeIntsIndexedWriter create( - final IOPeon ioPeon, - final String filenameBase, - final int maxValue, - final CompressedObjectStrategy.CompressionStrategy compression - ) - { - return new CompressedVSizeIntsIndexedWriter( - ioPeon, - filenameBase, - maxValue, - CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), - IndexIO.BYTE_ORDER, compression - ); - } - private static int sizePer(int maxValue, int chunkFactor) { return chunkFactor * VSizeIndexedInts.getNumBytesForMax(maxValue) @@ -125,14 +132,16 @@ public class CompressedVSizeIntsIndexedWriter extends SingleValueIndexedIntsWrit @Override protected void addValue(int val) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = ByteBuffer.allocate(chunkBytes).order(byteOrder); - endBuffer.limit(numBytes * chunkFactor); + flattener.write(endBuffer); + endBuffer.clear(); } intBuffer.putInt(0, val); - if (byteOrder.equals(ByteOrder.BIG_ENDIAN)) { + if (isBigEndian) { endBuffer.put(intBuffer.array(), Ints.BYTES - numBytes, numBytes); } else { endBuffer.put(intBuffer.array(), 0, numBytes); @@ -141,39 +150,28 @@ public class CompressedVSizeIntsIndexedWriter extends SingleValueIndexedIntsWrit } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - try { - if (numInserted > 0) { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); + writeEndBuffer(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + writeEndBuffer(); + metaSerdeHelper.writeTo(channel, this); + flattener.writeTo(channel, smoosher); + } + + private void writeEndBuffer() throws IOException + { + if (endBuffer != null) { + endBuffer.flip(); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); } endBuffer = null; } - finally { - flattener.close(); - } - } - - @Override - public long getSerializedSize() - { - return 1 + // version - 1 + // numBytes - Ints.BYTES + // numInserted - Ints.BYTES + // chunkFactor - 1 + // compression id - flattener.getSerializedSize(); - } - - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(numInserted))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(chunkFactor))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - flattener.writeToChannel(channel, smoosher); } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java index 2eae56128a9..1e6a2ea8a00 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java @@ -25,16 +25,18 @@ import com.google.common.base.Supplier; import com.google.common.collect.Maps; import io.druid.java.util.common.IAE; import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; -import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Map; +import java.util.function.Function; /** - * Compression of metrics is done by using a combination of {@link CompressedObjectStrategy.CompressionStrategy} + * Compression of metrics is done by using a combination of {@link 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. @@ -215,7 +217,7 @@ public class CompressionFactory /** * 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 + * {@link #setOutputStream(WriteOutBytes)} 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 @@ -226,7 +228,7 @@ public class CompressionFactory */ void setBuffer(ByteBuffer buffer); - void setOutputStream(OutputStream output); + void setOutputStream(WriteOutBytes output); void write(long value) throws IOException; @@ -239,7 +241,9 @@ public class CompressionFactory * 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; + void putMeta(ByteBuffer metaOut, CompressionStrategy strategy) throws IOException; + + int metaSize(); /** * Get the number of values that can be encoded into each block for the given block size in bytes @@ -252,14 +256,33 @@ public class CompressionFactory int getNumBytes(int values); } + static MetaSerdeHelper.FieldWriter longEncodingWriter( + Function getWriter, + Function getCompressionStrategy + ) + { + return new MetaSerdeHelper.FieldWriter() + { + @Override + public void writeTo(ByteBuffer buffer, T x) throws IOException + { + getWriter.apply(x).putMeta(buffer, getCompressionStrategy.apply(x)); + } + + @Override + public int size(T x) + { + return getWriter.apply(x).metaSize(); + } + }; + } + public interface LongEncodingReader { void setBuffer(ByteBuffer buffer); long read(int index); - int getNumBytes(int values); - LongEncodingReader duplicate(); } @@ -269,11 +292,10 @@ public class CompressionFactory ByteBuffer fromBuffer, ByteOrder order, LongEncodingFormat encodingFormat, - CompressedObjectStrategy.CompressionStrategy strategy, - SmooshedFileMapper fileMapper + CompressionStrategy strategy ) { - if (strategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + if (strategy == CompressionStrategy.NONE) { return new EntireLayoutIndexedLongSupplier(totalSize, encodingFormat.getReader(fromBuffer, order)); } else { return new BlockLayoutIndexedLongSupplier( @@ -282,28 +304,31 @@ public class CompressionFactory fromBuffer, order, encodingFormat.getReader(fromBuffer, order), - strategy, - fileMapper + strategy ); } } public static LongSupplierSerializer getLongSerializer( - IOPeon ioPeon, String filenameBase, ByteOrder order, + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + ByteOrder order, LongEncodingStrategy encodingStrategy, - CompressedObjectStrategy.CompressionStrategy compressionStrategy + CompressionStrategy compressionStrategy ) { if (encodingStrategy == LongEncodingStrategy.AUTO) { - return new IntermediateLongSupplierSerializer(ioPeon, filenameBase, order, compressionStrategy); + return new IntermediateLongSupplierSerializer(segmentWriteOutMedium, filenameBase, order, compressionStrategy); } else if (encodingStrategy == LongEncodingStrategy.LONGS) { - if (compressionStrategy == CompressedObjectStrategy.CompressionStrategy.NONE) { - return new EntireLayoutLongSupplierSerializer( - ioPeon, filenameBase, new LongsLongEncodingWriter(order) - ); + if (compressionStrategy == CompressionStrategy.NONE) { + return new EntireLayoutLongSupplierSerializer(segmentWriteOutMedium, new LongsLongEncodingWriter(order)); } else { return new BlockLayoutLongSupplierSerializer( - ioPeon, filenameBase, order, new LongsLongEncodingWriter(order), compressionStrategy + segmentWriteOutMedium, + filenameBase, + order, + new LongsLongEncodingWriter(order), + compressionStrategy ); } } else { @@ -318,30 +343,27 @@ public class CompressionFactory int sizePer, ByteBuffer fromBuffer, ByteOrder order, - CompressedObjectStrategy.CompressionStrategy strategy, - SmooshedFileMapper fileMapper + CompressionStrategy strategy ) { - if (strategy == CompressedObjectStrategy.CompressionStrategy.NONE) { + if (strategy == CompressionStrategy.NONE) { return new EntireLayoutIndexedFloatSupplier(totalSize, fromBuffer, order); } else { - return new BlockLayoutIndexedFloatSupplier(totalSize, sizePer, fromBuffer, order, strategy, fileMapper); + return new BlockLayoutIndexedFloatSupplier(totalSize, sizePer, fromBuffer, order, strategy); } } public static FloatSupplierSerializer getFloatSerializer( - IOPeon ioPeon, String filenameBase, ByteOrder order, - CompressedObjectStrategy.CompressionStrategy compressionStrategy + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + ByteOrder order, + CompressionStrategy compressionStrategy ) { - if (compressionStrategy == CompressedObjectStrategy.CompressionStrategy.NONE) { - return new EntireLayoutFloatSupplierSerializer( - ioPeon, filenameBase, order - ); + if (compressionStrategy == CompressionStrategy.NONE) { + return new EntireLayoutFloatSupplierSerializer(segmentWriteOutMedium, order); } else { - return new BlockLayoutFloatSupplierSerializer( - ioPeon, filenameBase, order, compressionStrategy - ); + return new BlockLayoutFloatSupplierSerializer(segmentWriteOutMedium, filenameBase, order, compressionStrategy); } } @@ -350,29 +372,29 @@ public class CompressionFactory int sizePer, ByteBuffer fromBuffer, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy strategy, - SmooshedFileMapper fileMapper + CompressionStrategy strategy ) { switch (strategy) { case NONE: return new EntireLayoutIndexedDoubleSupplier(totalSize, fromBuffer, byteOrder); default: - return new BlockLayoutIndexedDoubleSupplier(totalSize, sizePer, fromBuffer, byteOrder, strategy, fileMapper); + return new BlockLayoutIndexedDoubleSupplier(totalSize, sizePer, fromBuffer, byteOrder, strategy); } } + public static DoubleSupplierSerializer getDoubleSerializer( - IOPeon ioPeon, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - if (compression == CompressedObjectStrategy.CompressionStrategy.NONE) { - return new EntireLayoutDoubleSupplierSerializer(ioPeon, filenameBase, byteOrder); + if (compression == CompressionStrategy.NONE) { + return new EntireLayoutDoubleSupplierSerializer(segmentWriteOutMedium, byteOrder); } else { - return new BlockLayoutDoubleSupplierSerializer(ioPeon, filenameBase, byteOrder, compression); + return new BlockLayoutDoubleSupplierSerializer(segmentWriteOutMedium, filenameBase, byteOrder, compression); } } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java new file mode 100644 index 00000000000..c4a6fd742be --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java @@ -0,0 +1,338 @@ +/* + * 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.collect.Maps; +import com.ning.compress.BufferRecycler; +import com.ning.compress.lzf.LZFDecoder; +import com.ning.compress.lzf.LZFEncoder; +import io.druid.collections.ResourceHolder; +import io.druid.java.util.common.ByteBufferUtils; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.io.Closer; +import io.druid.segment.CompressedPools; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4SafeDecompressor; +import org.apache.commons.lang.ArrayUtils; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; + +/** + * 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 enum CompressionStrategy +{ + LZF((byte) 0x0) { + @Override + public Decompressor getDecompressor() + { + return LZFDecompressor.defaultDecompressor; + } + + @Override + public Compressor getCompressor() + { + return LZFCompressor.defaultCompressor; + } + }, + + LZ4((byte) 0x1) { + @Override + public Decompressor getDecompressor() + { + return LZ4Decompressor.defaultDecompressor; + } + + @Override + public Compressor getCompressor() + { + return LZ4Compressor.defaultCompressor; + } + }, + UNCOMPRESSED((byte) 0xFF) { + @Override + public Decompressor getDecompressor() + { + return UncompressedDecompressor.defaultDecompressor; + } + + @Override + 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"); + } + }; + public static final CompressionStrategy DEFAULT_COMPRESSION_STRATEGY = LZ4; + + final byte id; + + CompressionStrategy(byte id) + { + this.id = id; + } + + public byte getId() + { + return id; + } + + public abstract Compressor getCompressor(); + + public abstract Decompressor getDecompressor(); + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + public static CompressionStrategy fromString(String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } + + static final Map idMap = Maps.newHashMap(); + + static { + for (CompressionStrategy strategy : CompressionStrategy.values()) { + idMap.put(strategy.getId(), strategy); + } + } + + 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 interface Decompressor + { + /** + * Implementations of this method are expected to call out.flip() after writing to the output buffer + */ + void decompress(ByteBuffer in, int numBytes, ByteBuffer out); + } + + public static abstract class Compressor + { + /** + * Allocates a buffer that should be passed to {@link #compress} method as input buffer. Different Compressors + * require (or work more efficiently with) different kinds of buffers. + * + * If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer. + */ + ByteBuffer allocateInBuffer(int inputSize, Closer closer) + { + return ByteBuffer.allocate(inputSize); + } + + /** + * Allocates a buffer that should be passed to {@link #compress} method as output buffer. Different Compressors + * require (or work more efficiently with) different kinds of buffers. + * + * Allocates a buffer that is always enough to compress a byte sequence of the given size. + * + * If the allocated buffer is a direct buffer, it should be registered to be freed with the given Closer. + */ + abstract ByteBuffer allocateOutBuffer(int inputSize, Closer closer); + + /** + * Returns a ByteBuffer with compressed contents of in between it's position and limit. It may be the provided out + * ByteBuffer, or the in ByteBuffer, depending on the implementation. {@code out}'s position and limit + * are not respected and could be discarded. + * + *

Contents of {@code in} between it's position and limit are compressed. It's contents, position and limit + * shouldn't be changed in compress() method. + */ + public abstract ByteBuffer compress(ByteBuffer in, ByteBuffer out); + } + + public static class UncompressedCompressor extends Compressor + { + private static final UncompressedCompressor defaultCompressor = new UncompressedCompressor(); + + @Override + ByteBuffer allocateOutBuffer(int inputSize, Closer closer) + { + return ByteBuffer.allocate(inputSize); + } + + @Override + public ByteBuffer compress(ByteBuffer in, ByteBuffer out) + { + return in; + } + } + + public static class UncompressedDecompressor implements Decompressor + { + private static final UncompressedDecompressor defaultDecompressor = new UncompressedDecompressor(); + + @Override + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) + { + final ByteBuffer copyBuffer = in.duplicate(); + copyBuffer.limit(copyBuffer.position() + numBytes); + out.put(copyBuffer).flip(); + in.position(in.position() + numBytes); + } + + } + + public static class LZFDecompressor implements Decompressor + { + private static final LZFDecompressor defaultDecompressor = new LZFDecompressor(); + + @Override + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) + { + final byte[] bytes = new byte[numBytes]; + in.get(bytes); + + try (final ResourceHolder outputBytesHolder = CompressedPools.getOutputBytes()) { + final byte[] outputBytes = outputBytesHolder.get(); + final int numDecompressedBytes = LZFDecoder.decode(bytes, outputBytes); + out.put(outputBytes, 0, numDecompressedBytes); + out.flip(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + } + + public static class LZFCompressor extends Compressor + { + private static final LZFCompressor defaultCompressor = new LZFCompressor(); + + @Override + public ByteBuffer allocateOutBuffer(int inputSize, Closer closer) + { + return ByteBuffer.allocate(LZFEncoder.estimateMaxWorkspaceSize(inputSize)); + } + + @Override + public ByteBuffer compress(ByteBuffer in, ByteBuffer out) + { + try (final ResourceHolder bufferRecycler = CompressedPools.getBufferRecycler()) { + int encodedLen = LZFEncoder.appendEncoded( + in.array(), + in.arrayOffset() + in.position(), + in.remaining(), + out.array(), + out.arrayOffset(), + bufferRecycler.get() + ); + out.clear(); + out.limit(encodedLen); + return out; + } + } + } + + public static class LZ4Decompressor implements Decompressor + { + private static final LZ4SafeDecompressor lz4Safe = LZ4Factory.fastestInstance().safeDecompressor(); + private static final LZ4Decompressor defaultDecompressor = new LZ4Decompressor(); + + @Override + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) + { + // Since decompressed size is NOT known, must use lz4Safe + // lz4Safe.decompress does not modify buffer positions + final int numDecompressedBytes = lz4Safe.decompress( + in, + in.position(), + numBytes, + out, + out.position(), + out.remaining() + ); + out.limit(out.position() + numDecompressedBytes); + } + + } + + public static class LZ4Compressor extends Compressor + { + private static final LZ4Compressor defaultCompressor = new LZ4Compressor(); + private static final net.jpountz.lz4.LZ4Compressor lz4High = LZ4Factory.fastestInstance().highCompressor(); + + @Override + ByteBuffer allocateInBuffer(int inputSize, Closer closer) + { + ByteBuffer inBuffer = ByteBuffer.allocateDirect(inputSize); + closer.register(() -> ByteBufferUtils.free(inBuffer)); + return inBuffer; + } + + @Override + ByteBuffer allocateOutBuffer(int inputSize, Closer closer) + { + ByteBuffer outBuffer = ByteBuffer.allocateDirect(lz4High.maxCompressedLength(inputSize)); + closer.register(() -> ByteBufferUtils.free(outBuffer)); + return outBuffer; + } + + @Override + public ByteBuffer compress(ByteBuffer in, ByteBuffer out) + { + out.clear(); + int position = in.position(); + lz4High.compress(in, out); + in.position(position); + out.flip(); + return out; + } + } +} diff --git a/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java b/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java index fe9b2eceef9..044892d9f24 100644 --- a/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java +++ b/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java @@ -46,8 +46,7 @@ public class ConciseBitmapSerdeFactory implements BitmapSerdeFactory return bitmapFactory; } - private static class ImmutableConciseSetObjectStrategy - implements ObjectStrategy + private static class ImmutableConciseSetObjectStrategy implements ObjectStrategy { @Override public Class getClazz() diff --git a/processing/src/main/java/io/druid/segment/data/DecompressingByteBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/DecompressingByteBufferObjectStrategy.java new file mode 100644 index 00000000000..ff7a1426d17 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/DecompressingByteBufferObjectStrategy.java @@ -0,0 +1,84 @@ +/* + * 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 io.druid.collections.ResourceHolder; +import io.druid.segment.CompressedPools; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class DecompressingByteBufferObjectStrategy implements ObjectStrategy> +{ + private final ByteOrder order; + private final CompressionStrategy.Decompressor decompressor; + + DecompressingByteBufferObjectStrategy(ByteOrder order, CompressionStrategy compression) + { + this.order = order; + this.decompressor = compression.getDecompressor(); + } + + @Override + @SuppressWarnings("unchecked") + public Class> getClazz() + { + return (Class) ResourceHolder.class; + } + + @Override + public ResourceHolder fromByteBuffer(ByteBuffer buffer, int numBytes) + { + final ResourceHolder bufHolder = CompressedPools.getByteBuf(order); + final ByteBuffer buf = bufHolder.get(); + buf.clear(); + + decompressor.decompress(buffer, numBytes, buf); + // Needed, because if e. g. if this compressed buffer contains 3-byte integers, it should be possible to getInt() + // from the buffer, including padding. See CompressedVSizeIntsIndexedSupplier.bufferPadding(). + buf.limit(buf.capacity()); + return new ResourceHolder() + { + @Override + public ByteBuffer get() + { + return buf; + } + + @Override + public void close() + { + bufHolder.close(); + } + }; + } + + @Override + public int compare(ResourceHolder o1, ResourceHolder o2) + { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] toBytes(ResourceHolder holder) + { + throw new UnsupportedOperationException(); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java index 5c21615c914..a04b1b9b62f 100644 --- a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java +++ b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingReader.java @@ -65,12 +65,6 @@ public class DeltaLongEncodingReader implements CompressionFactory.LongEncodingR return base + deserializer.get(index); } - @Override - public int getNumBytes(int values) - { - return VSizeLongSerde.getSerializedSize(bitsPerValue, values); - } - @Override public CompressionFactory.LongEncodingReader duplicate() { diff --git a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java index e73bd72f472..7cefc0589fa 100644 --- a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java +++ b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java @@ -21,9 +21,9 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; -import java.io.OutputStream; import java.nio.ByteBuffer; public class DeltaLongEncodingWriter implements CompressionFactory.LongEncodingWriter @@ -46,7 +46,7 @@ public class DeltaLongEncodingWriter implements CompressionFactory.LongEncodingW } @Override - public void setOutputStream(OutputStream output) + public void setOutputStream(WriteOutBytes output) { serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); } @@ -58,13 +58,19 @@ public class DeltaLongEncodingWriter implements CompressionFactory.LongEncodingW } @Override - public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException + public void putMeta(ByteBuffer metaOut, 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)); + metaOut.put(CompressionFactory.setEncodingFlag(strategy.getId())); + metaOut.put(CompressionFactory.LongEncodingFormat.DELTA.getId()); + metaOut.put(CompressionFactory.DELTA_ENCODING_VERSION); + metaOut.putLong(base); + metaOut.putInt(bitsPerValue); + } + + @Override + public int metaSize() + { + return 1 + 1 + 1 + Longs.BYTES + Ints.BYTES; } @Override diff --git a/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java index 8589ca90ad0..fe804330ed6 100644 --- a/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java @@ -19,17 +19,12 @@ package io.druid.segment.data; +import io.druid.segment.serde.Serializer; -import io.druid.java.util.common.io.smoosh.FileSmoosher; - -import java.io.Closeable; import java.io.IOException; -import java.nio.channels.WritableByteChannel; -public interface DoubleSupplierSerializer extends Closeable +public interface DoubleSupplierSerializer extends Serializer { void open() throws IOException; void add(double value) throws IOException; - long getSerializedSize(); - void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java index 198fffd9835..563eff608b6 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java @@ -19,45 +19,42 @@ package io.druid.segment.data; -import com.google.common.io.ByteStreams; -import com.google.common.io.CountingOutputStream; -import com.google.common.primitives.Doubles; -import com.google.common.primitives.Ints; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; -import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; public class EntireLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer { - private final IOPeon ioPeon; - private final String valueFile; - private final String metaFile; - private CountingOutputStream valuesOut; - private long metaCount = 0; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((EntireLayoutDoubleSupplierSerializer x) -> CompressedDoublesIndexedSupplier.VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> 0) + .writeByte(x -> CompressionStrategy.NONE.getId()); + private final SegmentWriteOutMedium segmentWriteOutMedium; private final ByteBuffer orderBuffer; + private WriteOutBytes valuesOut; private int numInserted = 0; - public EntireLayoutDoubleSupplierSerializer(IOPeon ioPeon, String filenameBase, ByteOrder order) + public EntireLayoutDoubleSupplierSerializer(SegmentWriteOutMedium segmentWriteOutMedium, ByteOrder order) { - this.ioPeon = ioPeon; - this.valueFile = filenameBase + ".value"; - this.metaFile = filenameBase + ".format"; - this.orderBuffer = ByteBuffer.allocate(Doubles.BYTES); + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.orderBuffer = ByteBuffer.allocate(Double.BYTES); orderBuffer.order(order); } @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override @@ -67,38 +64,18 @@ public class EntireLayoutDoubleSupplierSerializer implements DoubleSupplierSeria orderBuffer.putDouble(value); valuesOut.write(orderBuffer.array()); ++numInserted; - } @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { - return metaCount + valuesOut.getCount(); + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override - public void writeToChannel( - WritableByteChannel channel, FileSmoosher smoosher - ) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) 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); - } - } - - @Override - public void close() throws IOException - { - valuesOut.close(); - try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { - metaOut.write(CompressedDoublesIndexedSupplier.version); - metaOut.write(Ints.toByteArray(numInserted)); - metaOut.write(Ints.toByteArray(0)); - metaOut.write(CompressedObjectStrategy.CompressionStrategy.NONE.getId()); - metaOut.close(); - metaCount = metaOut.getCount(); - } + metaSerdeHelper.writeTo(channel, this); + valuesOut.writeTo(channel); } } diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java index e49c32f9f69..8595d0121d7 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java @@ -19,49 +19,39 @@ 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 io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.serde.MetaSerdeHelper; 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 static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((EntireLayoutFloatSupplierSerializer x) -> CompressedFloatsIndexedSupplier.VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> 0) + .writeByte(x -> CompressionStrategy.NONE.getId()); - private final ByteBuffer orderBuffer; + private final boolean isLittleEndian; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private WriteOutBytes valuesOut; private int numInserted = 0; - public EntireLayoutFloatSupplierSerializer( - IOPeon ioPeon, String filenameBase, ByteOrder order - ) + EntireLayoutFloatSupplierSerializer(SegmentWriteOutMedium segmentWriteOutMedium, ByteOrder order) { - this.ioPeon = ioPeon; - this.valueFile = filenameBase + ".value"; - this.metaFile = filenameBase + ".format"; - - orderBuffer = ByteBuffer.allocate(Floats.BYTES); - orderBuffer.order(order); + this.segmentWriteOutMedium = segmentWriteOutMedium; + isLittleEndian = order.equals(ByteOrder.LITTLE_ENDIAN); } @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override @@ -73,51 +63,25 @@ public class EntireLayoutFloatSupplierSerializer implements FloatSupplierSeriali @Override public void add(float value) throws IOException { - orderBuffer.rewind(); - orderBuffer.putFloat(value); - valuesOut.write(orderBuffer.array()); + int valueBits = Float.floatToRawIntBits(value); + // WriteOutBytes are always big-endian, so need to reverse bytes + if (isLittleEndian) { + valueBits = Integer.reverseBytes(valueBits); + } + valuesOut.writeInt(valueBits); ++numInserted; } @Override - public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + public long getSerializedSize() 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); - } + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) 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, FileSmoosher smoosher) 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); - } + metaSerdeHelper.writeTo(channel, this); + valuesOut.writeTo(channel); } } diff --git a/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java index 2804fed9bf9..847fbcd89ad 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java @@ -19,46 +19,41 @@ 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.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; public class EntireLayoutLongSupplierSerializer implements LongSupplierSerializer { + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((EntireLayoutLongSupplierSerializer x) -> CompressedLongsIndexedSupplier.VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> 0) + .writeSomething(CompressionFactory.longEncodingWriter(x -> x.writer, x -> CompressionStrategy.NONE)); - 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 final SegmentWriteOutMedium segmentWriteOutMedium; + private WriteOutBytes valuesOut; private int numInserted = 0; - public EntireLayoutLongSupplierSerializer( - IOPeon ioPeon, - String filenameBase, + EntireLayoutLongSupplierSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, CompressionFactory.LongEncodingWriter writer ) { - this.ioPeon = ioPeon; - this.valueFile = filenameBase + ".value"; - this.metaFile = filenameBase + ".format"; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.writer = writer; } @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); writer.setOutputStream(valuesOut); } @@ -76,45 +71,17 @@ public class EntireLayoutLongSupplierSerializer implements LongSupplierSerialize } @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 + public long getSerializedSize() 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(); - } + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override - public long getSerializedSize() + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - return metaCount + valuesOut.getCount(); - } - - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) 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); - } + writer.flush(); + metaSerdeHelper.writeTo(channel, this); + valuesOut.writeTo(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 index d6cf71157b6..993ab2c43ce 100644 --- a/processing/src/main/java/io/druid/segment/data/FloatSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/FloatSupplierSerializer.java @@ -19,19 +19,13 @@ package io.druid.segment.data; -import com.google.common.io.ByteSink; -import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.Serializer; -import java.io.Closeable; import java.io.IOException; -import java.nio.channels.WritableByteChannel; -public interface FloatSupplierSerializer extends Closeable +public interface FloatSupplierSerializer extends Serializer { void open() throws IOException; int size(); void add(float value) throws IOException; - void closeAndConsolidate(ByteSink consolidatedOut) throws IOException; - long getSerializedSize(); - void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java index e5a4f9a9bd3..2bbfa9ef656 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -19,15 +19,22 @@ package io.druid.segment.data; +import com.google.common.base.Strings; import com.google.common.primitives.Ints; +import io.druid.collections.ResourceHolder; import io.druid.common.utils.SerializerUtils; -import io.druid.io.ZeroCopyByteArrayOutputStream; +import io.druid.io.Channels; import io.druid.java.util.common.IAE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Comparators; +import io.druid.java.util.common.io.Closer; +import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.serde.Serializer; import it.unimi.dsi.fastutil.bytes.ByteArrays; import java.io.Closeable; @@ -67,12 +74,19 @@ import java.util.Iterator; * value files are identified as: StringUtils.format("%s_value_%d", columnName, fileNumber) * number of value files == numElements/numberOfElementsPerValueFile */ -public class GenericIndexed implements Indexed +public class GenericIndexed implements Indexed, Serializer { static final byte VERSION_ONE = 0x1; static final byte VERSION_TWO = 0x2; static final byte REVERSE_LOOKUP_ALLOWED = 0x1; static final byte REVERSE_LOOKUP_DISALLOWED = 0x0; + + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((GenericIndexed x) -> VERSION_ONE) + .writeByte(x -> x.allowReverseLookup ? REVERSE_LOOKUP_ALLOWED : REVERSE_LOOKUP_DISALLOWED) + .writeInt(x -> Ints.checkedCast(x.theBuffer.remaining() + (long) Integer.BYTES)) + .writeInt(x -> x.size); + private static final SerializerUtils SERIALIZER_UTILS = new SerializerUtils(); public static final ObjectStrategy STRING_STRATEGY = new CacheableObjectStrategy() @@ -92,7 +106,7 @@ public class GenericIndexed implements Indexed @Override public byte[] toBytes(String val) { - if (val == null) { + if (Strings.isNullOrEmpty(val)) { return ByteArrays.EMPTY_ARRAY; } return StringUtils.toUtf8(val); @@ -138,9 +152,25 @@ public class GenericIndexed implements Indexed return fromIterable(Arrays.asList(objects), strategy); } + static GenericIndexed> ofCompressedByteBuffers( + Iterable buffers, + CompressionStrategy compression, + int bufferSize, + ByteOrder order, + Closer closer + ) + { + return fromIterableVersionOne( + buffers, + GenericIndexedWriter.compressedByteBuffersWriteObjectStrategy(compression, bufferSize, closer), + false, + new DecompressingByteBufferObjectStrategy(order, compression) + ); + } + public static GenericIndexed fromIterable(Iterable objectsIterable, ObjectStrategy strategy) { - return fromIterableVersionOne(objectsIterable, strategy); + return fromIterableVersionOne(objectsIterable, strategy, true, strategy); } static int getNumberOfFilesRequired(int bagSize, long numWritten) @@ -308,6 +338,7 @@ public class GenericIndexed implements Indexed return IndexedIterable.create(this).iterator(); } + @Override public long getSerializedSize() { if (!versionOne) { @@ -316,10 +347,11 @@ public class GenericIndexed implements Indexed return getSerializedSizeVersionOne(); } - public void writeToChannel(WritableByteChannel channel) throws IOException + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { if (versionOne) { - writeToChannelVersionOne(channel); + writeToVersionOne(channel); } else { throw new UnsupportedOperationException( "GenericIndexed serialization for V2 is unsupported. Use GenericIndexedWriter instead."); @@ -437,23 +469,25 @@ public class GenericIndexed implements Indexed ); } - private static GenericIndexed fromIterableVersionOne(Iterable objectsIterable, ObjectStrategy strategy) + private static GenericIndexed fromIterableVersionOne( + Iterable objectsIterable, + ObjectStrategy strategy, + boolean allowReverseLookup, + ObjectStrategy resultObjectStrategy + ) { Iterator objects = objectsIterable.iterator(); if (!objects.hasNext()) { final ByteBuffer buffer = ByteBuffer.allocate(Ints.BYTES).putInt(0); buffer.flip(); - return new GenericIndexed<>(buffer, strategy, true); + return new GenericIndexed<>(buffer, resultObjectStrategy, allowReverseLookup); } - boolean allowReverseLookup = true; int count = 0; - ZeroCopyByteArrayOutputStream headerBytes = new ZeroCopyByteArrayOutputStream(); - ZeroCopyByteArrayOutputStream valueBytes = new ZeroCopyByteArrayOutputStream(); - ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); + HeapByteBufferWriteOutBytes headerOut = new HeapByteBufferWriteOutBytes(); + HeapByteBufferWriteOutBytes valuesOut = new HeapByteBufferWriteOutBytes(); try { - int offset = 0; T prevVal = null; do { count++; @@ -462,11 +496,10 @@ public class GenericIndexed implements Indexed allowReverseLookup = false; } - final byte[] bytes = strategy.toBytes(next); - offset += Ints.BYTES + bytes.length; - SerializerUtils.writeBigEndianIntToOutputStream(headerBytes, offset, helperBuffer); - SerializerUtils.writeBigEndianIntToOutputStream(valueBytes, bytes.length, helperBuffer); - valueBytes.write(bytes); + // for compatibility with the format, but this field is unused + valuesOut.writeInt(0); + strategy.writeTo(next, valuesOut); + headerOut.writeInt(Ints.checkedCast(valuesOut.size())); if (prevVal instanceof Closeable) { CloseQuietly.close((Closeable) prevVal); @@ -482,22 +515,18 @@ public class GenericIndexed implements Indexed throw new RuntimeException(e); } - ByteBuffer theBuffer = ByteBuffer.allocate(Ints.BYTES + headerBytes.size() + valueBytes.size()); + ByteBuffer theBuffer = ByteBuffer.allocate(Ints.checkedCast(Ints.BYTES + headerOut.size() + valuesOut.size())); theBuffer.putInt(count); - headerBytes.writeTo(theBuffer); - valueBytes.writeTo(theBuffer); + headerOut.writeTo(theBuffer); + valuesOut.writeTo(theBuffer); theBuffer.flip(); - return new GenericIndexed<>(theBuffer.asReadOnlyBuffer(), strategy, allowReverseLookup); + return new GenericIndexed<>(theBuffer.asReadOnlyBuffer(), resultObjectStrategy, allowReverseLookup); } private long getSerializedSizeVersionOne() { - return theBuffer.remaining() - + 1 // version byte - + 1 // allowReverseLookup flag - + Ints.BYTES // numBytesUsed - + Ints.BYTES; // numElements + return metaSerdeHelper.size(this) + (long) theBuffer.remaining(); } private T getVersionOne(int index) @@ -552,15 +581,10 @@ public class GenericIndexed implements Indexed }; } - private void writeToChannelVersionOne(WritableByteChannel channel) throws IOException + private void writeToVersionOne(WritableByteChannel channel) throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{ - VERSION_ONE, - allowReverseLookup ? REVERSE_LOOKUP_ALLOWED : REVERSE_LOOKUP_DISALLOWED - })); - channel.write(ByteBuffer.wrap(Ints.toByteArray(theBuffer.remaining() + Ints.BYTES))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(size))); - channel.write(theBuffer.asReadOnlyBuffer()); + metaSerdeHelper.writeTo(channel, this); + Channels.writeFully(channel, theBuffer.asReadOnlyBuffer()); } 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 0f2c4636e26..29bea3d00a0 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -19,80 +19,151 @@ package io.druid.segment.data; -import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; -import com.google.common.io.ByteStreams; -import com.google.common.io.CountingOutputStream; -import com.google.common.io.InputSupplier; import com.google.common.primitives.Ints; -import com.google.common.primitives.Longs; -import io.druid.common.utils.SerializerUtils; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.serde.Serializer; +import it.unimi.dsi.fastutil.longs.LongArrayList; +import it.unimi.dsi.fastutil.longs.LongList; -import java.io.Closeable; -import java.io.File; -import java.io.FileInputStream; +import javax.annotation.Nullable; +import java.io.DataInput; +import java.io.DataInputStream; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; -import java.io.RandomAccessFile; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -import java.util.Arrays; /** * Streams arrays of objects out in the binary format described by GenericIndexed */ -public class GenericIndexedWriter implements Closeable +public class GenericIndexedWriter implements Serializer { private static int PAGE_SIZE = 4096; - private final IOPeon ioPeon; + + private static final MetaSerdeHelper singleFileMetaSerdeHelper = MetaSerdeHelper + .firstWriteByte((GenericIndexedWriter x) -> GenericIndexed.VERSION_ONE) + .writeByte( + x -> x.objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED + ) + .writeInt(x -> Ints.checkedCast(x.headerOut.size() + x.valuesOut.size() + Integer.BYTES)) + .writeInt(x -> x.numWritten); + + private static final MetaSerdeHelper multiFileMetaSerdeHelper = MetaSerdeHelper + .firstWriteByte((GenericIndexedWriter x) -> GenericIndexed.VERSION_TWO) + .writeByte( + x -> x.objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED + ) + .writeInt(GenericIndexedWriter::bagSizePower) + .writeInt(x -> x.numWritten) + .writeInt(x -> x.fileNameByteArray.length) + .writeByteArray(x -> x.fileNameByteArray); + + + static GenericIndexedWriter ofCompressedByteBuffers( + final SegmentWriteOutMedium segmentWriteOutMedium, + final String filenameBase, + final CompressionStrategy compressionStrategy, + final int bufferSize + ) + { + GenericIndexedWriter writer = new GenericIndexedWriter<>( + segmentWriteOutMedium, + filenameBase, + compressedByteBuffersWriteObjectStrategy(compressionStrategy, bufferSize, segmentWriteOutMedium.getCloser()) + ); + writer.objectsSorted = false; + return writer; + } + + static ObjectStrategy compressedByteBuffersWriteObjectStrategy( + final CompressionStrategy compressionStrategy, + final int bufferSize, + final Closer closer + ) + { + return new ObjectStrategy() + { + private final CompressionStrategy.Compressor compressor = compressionStrategy.getCompressor(); + private final ByteBuffer compressedDataBuffer = compressor.allocateOutBuffer(bufferSize, closer); + + @Override + public Class getClazz() + { + return ByteBuffer.class; + } + + @Override + public ByteBuffer fromByteBuffer(ByteBuffer buffer, int numBytes) + { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] toBytes(ByteBuffer val) + { + throw new UnsupportedOperationException(); + } + + @Override + public void writeTo(ByteBuffer val, WriteOutBytes out) throws IOException + { + compressedDataBuffer.clear(); + int valPos = val.position(); + out.write(compressor.compress(val, compressedDataBuffer)); + val.position(valPos); + } + + @Override + public int compare(ByteBuffer o1, ByteBuffer o2) + { + throw new UnsupportedOperationException(); + } + }; + } + + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ObjectStrategy strategy; private final int fileSizeLimit; private final byte[] fileNameByteArray; private boolean objectsSorted = true; private T prevObject = null; - private CountingOutputStream headerOut = null; - private CountingOutputStream valuesOut = null; - private CountingOutputStream headerOutLong = null; - private long numWritten = 0; + private WriteOutBytes headerOut = null; + private WriteOutBytes valuesOut = null; + private int numWritten = 0; private boolean requireMultipleFiles = false; - private ByteBuffer buf; - private final ByteBuffer sizeHelperBuffer = ByteBuffer.allocate(Ints.BYTES); + private LongList headerOutLong; + private final ByteBuffer getOffsetBuffer = ByteBuffer.allocate(Integer.BYTES); - public GenericIndexedWriter( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) + public GenericIndexedWriter(SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy) { - this(ioPeon, filenameBase, strategy, Integer.MAX_VALUE & ~PAGE_SIZE); + this(segmentWriteOutMedium, filenameBase, strategy, Integer.MAX_VALUE & ~PAGE_SIZE); } public GenericIndexedWriter( - IOPeon ioPeon, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy, int fileSizeLimit ) { - this.ioPeon = ioPeon; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.strategy = strategy; this.fileSizeLimit = fileSizeLimit; fileNameByteArray = StringUtils.toUtf8(filenameBase); - buf = ByteBuffer.allocate(Ints.BYTES); } public static String generateValueFileName(String fileNameBase, int fileNum) @@ -127,8 +198,8 @@ public class GenericIndexedWriter implements Closeable public void open() throws IOException { - headerOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("header"))); - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("values"))); + headerOut = segmentWriteOutMedium.makeWriteOutBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } public void setObjectsNotSorted() @@ -142,85 +213,149 @@ public class GenericIndexedWriter implements Closeable objectsSorted = false; } - byte[] bytesToWrite = strategy.toBytes(objectToWrite); - ++numWritten; - SerializerUtils.writeBigEndianIntToOutputStream(valuesOut, bytesToWrite.length, sizeHelperBuffer); - valuesOut.write(bytesToWrite); + // for compatibility with the format (see GenericIndexed javadoc for description of the format), but this field is + // unused. + valuesOut.writeInt(0); + strategy.writeTo(objectToWrite, valuesOut); if (!requireMultipleFiles) { - SerializerUtils.writeBigEndianIntToOutputStream(headerOut, Ints.checkedCast(valuesOut.getCount()), buf); + headerOut.writeInt(Ints.checkedCast(valuesOut.size())); } else { - SerializerUtils.writeNativeOrderedLongToOutputStream(headerOutLong, valuesOut.getCount(), buf); + headerOutLong.add(valuesOut.size()); } if (!requireMultipleFiles && getSerializedSize() > fileSizeLimit) { requireMultipleFiles = true; initializeHeaderOutLong(); - buf = ByteBuffer.allocate(Longs.BYTES).order(ByteOrder.nativeOrder()); } - prevObject = objectToWrite; + if (objectsSorted) { + prevObject = objectToWrite; + } } - private String makeFilename(String suffix) + @Nullable + public T get(int index) throws IOException { - return StringUtils.format("%s.%s", filenameBase, suffix); + long startOffset; + if (index == 0) { + startOffset = Integer.BYTES; + } else { + startOffset = getOffset(index - 1) + Integer.BYTES; + } + long endOffset = getOffset(index); + int valueSize = Ints.checkedCast(endOffset - startOffset); + if (valueSize == 0) { + return null; + } + ByteBuffer bb = ByteBuffer.allocate(valueSize); + valuesOut.readFully(startOffset, bb); + bb.clear(); + return strategy.fromByteBuffer(bb, valueSize); + } + + private long getOffset(int index) throws IOException + { + if (!requireMultipleFiles) { + getOffsetBuffer.clear(); + headerOut.readFully(index * (long) Integer.BYTES, getOffsetBuffer); + return getOffsetBuffer.getInt(0); + } else { + return headerOutLong.getLong(index); + } } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - valuesOut.close(); if (requireMultipleFiles) { - closeMultiFiles(); + // for multi-file version (version 2), getSerializedSize() returns number of bytes in meta file. + return multiFileMetaSerdeHelper.size(this); } else { - closeSingleFile(); + return singleFileMetaSerdeHelper.size(this) + headerOut.size() + valuesOut.size(); } } - private void closeSingleFile() throws IOException + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - headerOut.close(); - final long numBytesWritten = headerOut.getCount() + valuesOut.getCount(); + if (requireMultipleFiles) { + writeToMultiFiles(channel, smoosher); + } else { + writeToSingleFile(channel); + } + } + + private void writeToSingleFile(WritableByteChannel channel) throws IOException + { + final long numBytesWritten = headerOut.size() + valuesOut.size(); Preconditions.checkState( - headerOut.getCount() == (numWritten * 4), + headerOut.size() == (numWritten * 4), "numWritten[%s] number of rows should have [%s] bytes written to headerOut, had[%s]", numWritten, numWritten * 4, - headerOut.getCount() + headerOut.size() ); Preconditions.checkState( - numBytesWritten < fileSizeLimit, "Wrote[%s] bytes to base file %s, which is too many.", - numBytesWritten, - filenameBase + numBytesWritten < fileSizeLimit, "Wrote[%s] bytes, which is too many.", + numBytesWritten ); - try (OutputStream metaOut = ioPeon.makeOutputStream(makeFilename("meta"))) { - metaOut.write(GenericIndexed.VERSION_ONE); - metaOut.write(objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED); - metaOut.write(Ints.toByteArray(Ints.checkedCast(numBytesWritten + 4))); - metaOut.write(Ints.toByteArray(Ints.checkedCast(numWritten))); - } + singleFileMetaSerdeHelper.writeTo(channel, this); + headerOut.writeTo(channel); + valuesOut.writeTo(channel); } - private void closeMultiFiles() throws IOException + private void writeToMultiFiles(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - headerOutLong.close(); Preconditions.checkState( - headerOutLong.getCount() == (numWritten * Longs.BYTES), - "numWritten[%s] number of rows should have [%s] bytes written to headerOutLong, had[%s]", + headerOutLong.size() == numWritten, + "numWritten[%s] number of rows doesn't match headerOutLong's size[%s]", numWritten, - numWritten * Longs.BYTES, - headerOutLong.getCount() + headerOutLong.size() ); Preconditions.checkState( - headerOutLong.getCount() < (Integer.MAX_VALUE & ~PAGE_SIZE), - "Wrote[%s] bytes in header file of base file %s, which is too many.", - headerOutLong.getCount(), - filenameBase + (((long) headerOutLong.size()) * Long.BYTES) < (Integer.MAX_VALUE & ~PAGE_SIZE), + "Wrote[%s] bytes in header, which is too many.", + (((long) headerOutLong.size()) * Long.BYTES) ); + + if (smoosher == null) { + throw new IAE("version 2 GenericIndexedWriter requires FileSmoosher."); + } + + int bagSizePower = bagSizePower(); + multiFileMetaSerdeHelper.writeTo(channel, this); + + long previousValuePosition = 0; + int bagSize = 1 << bagSizePower; + + int numberOfFilesRequired = GenericIndexed.getNumberOfFilesRequired(bagSize, numWritten); + byte[] buffer = new byte[1 << 16]; + + try (InputStream is = valuesOut.asInputStream()) { + int counter = -1; + for (int i = 0; i < numberOfFilesRequired; i++) { + long valuePosition; + if (i != numberOfFilesRequired - 1) { + valuePosition = headerOutLong.getLong(bagSize + counter); + counter = counter + bagSize; + } else { + valuePosition = headerOutLong.getLong(numWritten - 1); + } + + long numBytesToPutInFile = valuePosition - previousValuePosition; + + try (SmooshedWriter smooshChannel = smoosher + .addWithSmooshedWriter(generateValueFileName(filenameBase, i), numBytesToPutInFile)) { + writeBytesIntoSmooshedChannel(numBytesToPutInFile, buffer, smooshChannel, is); + previousValuePosition = valuePosition; + } + } + } + writeHeaderLong(smoosher, bagSizePower); } /** @@ -232,25 +367,19 @@ public class GenericIndexedWriter implements Closeable */ private int bagSizePower() throws IOException { - long avgObjectSize = (valuesOut.getCount() + numWritten - 1) / numWritten; + long avgObjectSize = (valuesOut.size() + numWritten - 1) / numWritten; - File f = ioPeon.getFile(makeFilename("headerLong")); - Preconditions.checkNotNull(f, "header file missing."); - - try (RandomAccessFile headerFile = new RandomAccessFile(f, "r")) { - for (int i = 31; i >= 0; --i) { - if ((1L << i) * avgObjectSize <= fileSizeLimit) { - if (actuallyFits(i, headerFile)) { - return i; - } + for (int i = 31; i >= 0; --i) { + if ((1L << i) * avgObjectSize <= fileSizeLimit) { + if (actuallyFits(i)) { + return i; } } } throw new ISE( - "no value split found with fileSizeLimit [%d], avgObjectSize [%d] while serializing [%s]", + "no value split found with fileSizeLimit [%d], avgObjectSize [%d]", fileSizeLimit, - avgObjectSize, - filenameBase + avgObjectSize ); } @@ -258,17 +387,16 @@ public class GenericIndexedWriter implements Closeable * Checks if candidate value splits can divide value file in such a way no object/element crosses the value splits. * * @param powerTwo candidate value split expressed as power of 2. - * @param headerFile header file. * * @return true if candidate value split can hold all splits. * * @throws IOException */ - private boolean actuallyFits(int powerTwo, RandomAccessFile headerFile) throws IOException + private boolean actuallyFits(int powerTwo) throws IOException { long lastValueOffset = 0; long currentValueOffset = 0; - long valueBytesWritten = valuesOut.getCount(); + long valueBytesWritten = valuesOut.size(); long headerIndex = 0; long bagSize = 1L << powerTwo; @@ -277,11 +405,9 @@ public class GenericIndexedWriter implements Closeable if (headerIndex >= numWritten) { return true; } else if (headerIndex + bagSize <= numWritten) { - headerFile.seek((headerIndex + bagSize - 1) * Longs.BYTES); - currentValueOffset = Long.reverseBytes(headerFile.readLong()); + currentValueOffset = headerOutLong.getLong(Ints.checkedCast(headerIndex + bagSize - 1)); } else if (numWritten < headerIndex + bagSize) { - headerFile.seek((numWritten - 1) * Longs.BYTES); - currentValueOffset = Long.reverseBytes(headerFile.readLong()); + currentValueOffset = headerOutLong.getLong(numWritten - 1); } if (currentValueOffset - lastValueOffset <= fileSizeLimit) { @@ -294,132 +420,17 @@ public class GenericIndexedWriter implements Closeable return true; } - public long getSerializedSize() - { - // for version 2 getSerializedSize() returns number of bytes in meta file. - if (!requireMultipleFiles) { - return 2 + // version and sorted flag - Ints.BYTES + // numBytesWritten - Ints.BYTES + // numElements - headerOut.getCount() + // header length - valuesOut.getCount(); // value length - } else { - return 2 + // version and sorted flag - Ints.BYTES + // numElements as log base 2. - Ints.BYTES + // number of files - Ints.BYTES + // column name Size - fileNameByteArray.length; - } - } - - @Deprecated - public InputSupplier combineStreams() - { - // ByteSource.concat is only available in guava 15 and higher - // This is guava 14 compatible - if (requireMultipleFiles) { - throw new ISE("Can not combine streams for version 2."); //fallback to old behaviour. - } - - return ByteStreams.join( - Iterables.transform( - Arrays.asList("meta", "header", "values"), - new Function>() - { - @Override - public InputSupplier apply(final String input) - { - return new InputSupplier() - { - @Override - public InputStream getInput() throws IOException - { - return ioPeon.makeInputStream(makeFilename(input)); - } - }; - } - } - ) - ); - } - - private void writeToChannelVersionOne(WritableByteChannel channel) throws IOException - { - try (ReadableByteChannel from = Channels.newChannel(combineStreams().getInput())) { - ByteStreams.copy(from, channel); - } - - } - - private void writeToChannelVersionTwo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - if (smoosher == null) { - throw new IAE("version 2 GenericIndexedWriter requires FileSmoosher."); - } - - int bagSizePower = bagSizePower(); - OutputStream metaOut = Channels.newOutputStream(channel); - metaOut.write(GenericIndexed.VERSION_TWO); - metaOut.write(objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED); - metaOut.write(Ints.toByteArray(bagSizePower)); - metaOut.write(Ints.toByteArray(Ints.checkedCast(numWritten))); - metaOut.write(Ints.toByteArray(fileNameByteArray.length)); - metaOut.write(fileNameByteArray); - - try (RandomAccessFile headerFile = new RandomAccessFile(ioPeon.getFile(makeFilename("headerLong")), "r")) { - Preconditions.checkNotNull(headerFile, "header file missing."); - long previousValuePosition = 0; - int bagSize = 1 << bagSizePower; - - int numberOfFilesRequired = GenericIndexed.getNumberOfFilesRequired(bagSize, numWritten); - byte[] buffer = new byte[1 << 16]; - - try (InputStream is = new FileInputStream(ioPeon.getFile(makeFilename("values")))) { - int counter = -1; - - for (int i = 0; i < numberOfFilesRequired; i++) { - if (i != numberOfFilesRequired - 1) { - headerFile.seek((bagSize + counter) * Longs.BYTES); // 8 for long bytes. - counter = counter + bagSize; - } else { - headerFile.seek((numWritten - 1) * Longs.BYTES); // for remaining items. - } - - long valuePosition = Long.reverseBytes(headerFile.readLong()); - long numBytesToPutInFile = valuePosition - previousValuePosition; - - try (SmooshedWriter smooshChannel = smoosher - .addWithSmooshedWriter(generateValueFileName(filenameBase, i), numBytesToPutInFile)) { - writeBytesIntoSmooshedChannel(numBytesToPutInFile, buffer, smooshChannel, is); - previousValuePosition = valuePosition; - } - } - } - writeHeaderLong(smoosher, headerFile, bagSizePower, buffer); - } - } - - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - if (!requireMultipleFiles) { - writeToChannelVersionOne(channel); - } else { - writeToChannelVersionTwo(channel, smoosher); - } - } - - private void writeHeaderLong(FileSmoosher smoosher, RandomAccessFile headerFile, int bagSizePower, byte[] buffer) + private void writeHeaderLong(FileSmoosher smoosher, int bagSizePower) throws IOException { ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES).order(ByteOrder.nativeOrder()); - try (CountingOutputStream finalHeaderOut = new CountingOutputStream( - ioPeon.makeOutputStream(makeFilename("header_final")))) { - int numberOfElementsPerValueFile = 1 << bagSizePower; - long currentNumBytes = 0; - long relativeRefBytes = 0; - long relativeNumBytes; - headerFile.seek(0); + int numberOfElementsPerValueFile = 1 << bagSizePower; + long currentNumBytes = 0; + long relativeRefBytes = 0; + long relativeNumBytes; + try (SmooshedWriter smooshChannel = smoosher + .addWithSmooshedWriter(generateHeaderFileName(filenameBase), numWritten * Integer.BYTES)) { // following block converts long header indexes into int header indexes. for (int pos = 0; pos < numWritten; pos++) { @@ -428,38 +439,22 @@ public class GenericIndexedWriter implements Closeable if ((pos & (numberOfElementsPerValueFile - 1)) == 0) { relativeRefBytes = currentNumBytes; } - currentNumBytes = Long.reverseBytes(headerFile.readLong()); + currentNumBytes = headerOutLong.getLong(pos); relativeNumBytes = currentNumBytes - relativeRefBytes; - SerializerUtils.writeNativeOrderedIntToOutputStream( - finalHeaderOut, - Ints.checkedCast(relativeNumBytes), - helperBuffer - ); + helperBuffer.putInt(0, Ints.checkedCast(relativeNumBytes)); + helperBuffer.clear(); + smooshChannel.write(helperBuffer); } - - long numBytesToPutInFile = finalHeaderOut.getCount(); - finalHeaderOut.close(); - try (InputStream is = new FileInputStream(ioPeon.getFile(makeFilename("header_final")))) { - try (SmooshedWriter smooshChannel = smoosher - .addWithSmooshedWriter(generateHeaderFileName(filenameBase), numBytesToPutInFile)) { - writeBytesIntoSmooshedChannel(numBytesToPutInFile, buffer, smooshChannel, is); - } - } - } } private void initializeHeaderOutLong() throws IOException { - headerOut.close(); - headerOutLong = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("headerLong"))); - - try (RandomAccessFile headerFile = new RandomAccessFile(ioPeon.getFile(makeFilename("header")), "r")) { - ByteBuffer buf = ByteBuffer.allocate(Longs.BYTES).order(ByteOrder.nativeOrder()); - for (int i = 0; i < numWritten; i++) { - int count = headerFile.readInt(); - SerializerUtils.writeNativeOrderedLongToOutputStream(headerOutLong, count, buf); - } + headerOutLong = new LongArrayList(); + DataInput headerOutAsIntInput = new DataInputStream(headerOut.asInputStream()); + for (int i = 0; i < numWritten; i++) { + int count = headerOutAsIntInput.readInt(); + headerOutLong.add(count); } } diff --git a/processing/src/main/java/io/druid/segment/data/ImmutableRTreeObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/ImmutableRTreeObjectStrategy.java index 8623b0a59c6..14f38f4ef49 100644 --- a/processing/src/main/java/io/druid/segment/data/ImmutableRTreeObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/ImmutableRTreeObjectStrategy.java @@ -22,6 +22,7 @@ package io.druid.segment.data; import com.google.common.collect.Ordering; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.spatial.ImmutableRTree; +import it.unimi.dsi.fastutil.bytes.ByteArrays; import java.nio.ByteBuffer; @@ -71,7 +72,7 @@ public class ImmutableRTreeObjectStrategy implements ObjectStrategy 0) { - delegate.add(tempIn.readLong()); - available--; - } + delegate.open(); + for (int i = 0; i < tempOut.size(); i++) { + delegate.add(tempOut.getLong(i)); } } @Override - public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException + public long getSerializedSize() throws IOException { - tempOut.close(); makeDelegate(); - delegate.closeAndConsolidate(consolidatedOut); - } - - @Override - public void close() throws IOException - { - tempOut.close(); - makeDelegate(); - delegate.close(); - } - - @Override - public long getSerializedSize() - { return delegate.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - delegate.writeToChannel(channel, smoosher); + makeDelegate(); + delegate.writeTo(channel, smoosher); } } diff --git a/processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java index b40554fddfe..d03fb94760c 100644 --- a/processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/LongSupplierSerializer.java @@ -19,21 +19,15 @@ package io.druid.segment.data; -import com.google.common.io.ByteSink; -import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.serde.Serializer; -import java.io.Closeable; import java.io.IOException; -import java.nio.channels.WritableByteChannel; /** */ -public interface LongSupplierSerializer extends Closeable +public interface LongSupplierSerializer extends Serializer { void open() throws IOException; int size(); void add(long value) throws IOException; - void closeAndConsolidate(ByteSink consolidatedOut) throws IOException; - long getSerializedSize(); - void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) 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 index 9cbb4f96a70..078226e1cfc 100644 --- a/processing/src/main/java/io/druid/segment/data/LongsLongEncodingReader.java +++ b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingReader.java @@ -19,8 +19,6 @@ package io.druid.segment.data; -import com.google.common.primitives.Longs; - import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.LongBuffer; @@ -51,12 +49,6 @@ public class LongsLongEncodingReader implements CompressionFactory.LongEncodingR return buffer.get(buffer.position() + index); } - @Override - public int getNumBytes(int values) - { - return values * Longs.BYTES; - } - @Override public CompressionFactory.LongEncodingReader 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 index 8a32d01e999..baa01879ae9 100644 --- a/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java +++ b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import com.google.common.primitives.Longs; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; import java.io.OutputStream; @@ -52,7 +53,7 @@ public class LongsLongEncodingWriter implements CompressionFactory.LongEncodingW } @Override - public void setOutputStream(OutputStream output) + public void setOutputStream(WriteOutBytes output) { outBuffer = null; outStream = output; @@ -77,9 +78,15 @@ public class LongsLongEncodingWriter implements CompressionFactory.LongEncodingW } @Override - public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException + public void putMeta(ByteBuffer metaOut, CompressionStrategy strategy) throws IOException { - metaOut.write(strategy.getId()); + metaOut.put(strategy.getId()); + } + + @Override + public int metaSize() + { + return 1; } @Override diff --git a/processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java b/processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java index e1d096b15ce..05d6a578045 100644 --- a/processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java +++ b/processing/src/main/java/io/druid/segment/data/MultiValueIndexedIntsWriter.java @@ -19,11 +19,11 @@ package io.druid.segment.data; -import com.google.common.primitives.Ints; import io.druid.java.util.common.IAE; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; import java.io.IOException; -import java.util.List; public abstract class MultiValueIndexedIntsWriter implements IndexedIntsWriter { @@ -33,13 +33,13 @@ public abstract class MultiValueIndexedIntsWriter implements IndexedIntsWriter if (obj == null) { addValues(null); } else if (obj instanceof int[]) { - addValues(Ints.asList((int[]) obj)); - } else if (obj instanceof List) { - addValues((List) obj); + addValues(IntArrayList.wrap((int[]) obj)); + } else if (obj instanceof IntList) { + addValues((IntList) obj); } else { throw new IAE("unsupported multi-value type: " + obj.getClass()); } } - protected abstract void addValues(List vals) throws IOException; + protected abstract void addValues(IntList vals) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java index c7b1f411d69..e5957a7ee94 100644 --- a/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java @@ -20,7 +20,9 @@ package io.druid.segment.data; import io.druid.guice.annotations.ExtensionPoint; +import io.druid.segment.writeout.WriteOutBytes; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Comparator; @@ -44,4 +46,9 @@ public interface ObjectStrategy extends Comparator */ T fromByteBuffer(ByteBuffer buffer, int numBytes); byte[] toBytes(T val); + + default void writeTo(T val, WriteOutBytes out) throws IOException + { + out.write(toBytes(val)); + } } diff --git a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java index d921eac1204..cb76d50189d 100644 --- a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java +++ b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -68,8 +68,7 @@ public class RoaringBitmapSerdeFactory implements BitmapSerdeFactory return bitmapFactory; } - private static class ImmutableRoaringBitmapObjectStrategy - implements ObjectStrategy + private static class ImmutableRoaringBitmapObjectStrategy implements ObjectStrategy { @Override public Class getClazz() diff --git a/processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java b/processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java index db79c23229b..bd1c10e029f 100644 --- a/processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java +++ b/processing/src/main/java/io/druid/segment/data/TableLongEncodingReader.java @@ -71,12 +71,6 @@ public class TableLongEncodingReader implements CompressionFactory.LongEncodingR return table[(int) deserializer.get(index)]; } - @Override - public int getNumBytes(int values) - { - return VSizeLongSerde.getSerializedSize(bitsPerValue, values); - } - @Override public CompressionFactory.LongEncodingReader duplicate() { diff --git a/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java index 69fde7e03fb..36382a2b13b 100644 --- a/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java +++ b/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java @@ -21,13 +21,12 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import io.druid.common.utils.SerializerUtils; import io.druid.java.util.common.IAE; +import io.druid.segment.writeout.WriteOutBytes; import it.unimi.dsi.fastutil.longs.Long2IntMap; import it.unimi.dsi.fastutil.longs.LongList; import java.io.IOException; -import java.io.OutputStream; import java.nio.ByteBuffer; public class TableLongEncodingWriter implements CompressionFactory.LongEncodingWriter @@ -55,7 +54,7 @@ public class TableLongEncodingWriter implements CompressionFactory.LongEncodingW } @Override - public void setOutputStream(OutputStream output) + public void setOutputStream(WriteOutBytes output) { serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); } @@ -75,18 +74,23 @@ public class TableLongEncodingWriter implements CompressionFactory.LongEncodingW } @Override - public void putMeta(OutputStream metaOut, CompressedObjectStrategy.CompressionStrategy strategy) throws IOException + public void putMeta(ByteBuffer metaOut, 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())); - ByteBuffer helperBuffer = ByteBuffer.allocate(Longs.BYTES); + metaOut.put(CompressionFactory.setEncodingFlag(strategy.getId())); + metaOut.put(CompressionFactory.LongEncodingFormat.TABLE.getId()); + metaOut.put(CompressionFactory.TABLE_ENCODING_VERSION); + metaOut.putInt(table.size()); for (int i = 0; i < valueAddedInOrder.size(); i++) { - SerializerUtils.writeBigEndianLongToOutputStream(metaOut, valueAddedInOrder.getLong(i), helperBuffer); + metaOut.putLong(valueAddedInOrder.getLong(i)); } } + @Override + public int metaSize() + { + return 1 + 1 + 1 + Ints.BYTES + (table.size() * Longs.BYTES); + } + @Override public int getBlockSize(int bytesPerBlock) { diff --git a/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java b/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java deleted file mode 100644 index ff22058f693..00000000000 --- a/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java +++ /dev/null @@ -1,95 +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.collect.Maps; - -import java.io.BufferedOutputStream; -import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Map; - -/** - */ -public class TmpFileIOPeon implements IOPeon -{ - private final File dir; - private final boolean allowOverwrite; - private final Map createdFiles = Maps.newLinkedHashMap(); - - public TmpFileIOPeon() - { - this(true); - } - - public TmpFileIOPeon(boolean allowOverwrite) - { - this(null, allowOverwrite); - } - - public TmpFileIOPeon(File dir, boolean allowOverwrite) - { - this.dir = dir; - this.allowOverwrite = allowOverwrite; - } - - @Override - public OutputStream makeOutputStream(String filename) throws IOException - { - File retFile = createdFiles.get(filename); - if (retFile == null) { - retFile = File.createTempFile("filePeon", filename, dir); - createdFiles.put(filename, retFile); - return new BufferedOutputStream(new FileOutputStream(retFile)); - } else if (allowOverwrite) { - return new BufferedOutputStream(new FileOutputStream(retFile)); - } else { - throw new IOException("tmp file conflicts, file[" + filename + "] already exist!"); - } - } - - @Override - public InputStream makeInputStream(String filename) throws IOException - { - final File retFile = createdFiles.get(filename); - - return retFile == null ? null : new FileInputStream(retFile); - } - - @Override - public void close() throws IOException - { - for (File file : createdFiles.values()) { - file.delete(); - } - createdFiles.clear(); - } - - @Override - public File getFile(String filename) - { - return createdFiles.get(filename); - } - -} diff --git a/processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java deleted file mode 100644 index 1595e2b9af3..00000000000 --- a/processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java +++ /dev/null @@ -1,88 +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 io.druid.java.util.common.guava.Comparators; - -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 Comparators.naturalNullsFirst().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/VSizeIndexed.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java index 16731a93bff..84db1835c26 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java @@ -20,11 +20,14 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; -import io.druid.common.utils.SerializerUtils; -import io.druid.io.ZeroCopyByteArrayOutputStream; +import io.druid.common.utils.ByteUtils; +import io.druid.io.Channels; import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -33,17 +36,22 @@ import java.util.Iterator; /** */ -public class VSizeIndexed implements IndexedMultivalue +public class VSizeIndexed implements IndexedMultivalue, WritableSupplier> { - private static final byte version = 0x1; + private static final byte VERSION = 0x1; + + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((VSizeIndexed x) -> VERSION) + .writeByte(x -> ByteUtils.checkedCast(x.numBytes)) + .writeInt(x -> Ints.checkedCast(x.theBuffer.remaining() + (long) Integer.BYTES)) + .writeInt(x -> x.size); public static VSizeIndexed fromIterable(Iterable objectsIterable) { Iterator objects = objectsIterable.iterator(); if (!objects.hasNext()) { - final ByteBuffer buffer = ByteBuffer.allocate(4).putInt(0); - buffer.flip(); - return new VSizeIndexed(buffer, 4); + final ByteBuffer buffer = ByteBuffer.allocate(Ints.BYTES).putInt(0, 0); + return new VSizeIndexed(buffer, Ints.BYTES); } int numBytes = -1; @@ -57,29 +65,27 @@ public class VSizeIndexed implements IndexedMultivalue ++count; } - ZeroCopyByteArrayOutputStream headerBytes = new ZeroCopyByteArrayOutputStream(4 + (count * 4)); - ZeroCopyByteArrayOutputStream valueBytes = new ZeroCopyByteArrayOutputStream(); - ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); + HeapByteBufferWriteOutBytes headerBytes = new HeapByteBufferWriteOutBytes(); + HeapByteBufferWriteOutBytes valueBytes = new HeapByteBufferWriteOutBytes(); int offset = 0; try { - SerializerUtils.writeBigEndianIntToOutputStream(headerBytes, count, helperBuffer); + headerBytes.writeInt(count); for (VSizeIndexedInts object : objectsIterable) { if (object.getNumBytes() != numBytes) { throw new ISE("val.numBytes[%s] != numBytesInValue[%s]", object.getNumBytes(), numBytes); } - byte[] bytes = object.getBytesNoPadding(); - offset += bytes.length; - SerializerUtils.writeBigEndianIntToOutputStream(headerBytes, offset, helperBuffer); - valueBytes.write(bytes); + offset += object.getNumBytesNoPadding(); + headerBytes.writeInt(offset); + object.writeBytesNoPaddingTo(valueBytes); } - valueBytes.write(new byte[4 - numBytes]); + valueBytes.write(new byte[Ints.BYTES - numBytes]); } catch (IOException e) { throw new RuntimeException(e); } - ByteBuffer theBuffer = ByteBuffer.allocate(headerBytes.size() + valueBytes.size()); + ByteBuffer theBuffer = ByteBuffer.allocate(Ints.checkedCast(headerBytes.size() + valueBytes.size())); headerBytes.writeTo(theBuffer); valueBytes.writeTo(theBuffer); theBuffer.flip(); @@ -149,24 +155,30 @@ public class VSizeIndexed implements IndexedMultivalue throw new UnsupportedOperationException("Reverse lookup not allowed."); } - public int getSerializedSize() + @Override + public long getSerializedSize() throws IOException { - return theBuffer.remaining() + 4 + 4 + 2; + return metaSerdeHelper.size(this) + (long) theBuffer.remaining(); } - public void writeToChannel(WritableByteChannel channel) throws IOException + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{version, (byte) numBytes})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(theBuffer.remaining() + 4))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(size))); - channel.write(theBuffer.asReadOnlyBuffer()); + metaSerdeHelper.writeTo(channel, this); + Channels.writeFully(channel, theBuffer.asReadOnlyBuffer()); + } + + @Override + public IndexedMultivalue get() + { + return this; } public static VSizeIndexed readFromByteBuffer(ByteBuffer buffer) { byte versionFromBuffer = buffer.get(); - if (version == versionFromBuffer) { + if (VERSION == versionFromBuffer) { int numBytes = buffer.get(); int size = buffer.getInt(); ByteBuffer bufferToUse = buffer.asReadOnlyBuffer(); @@ -196,37 +208,4 @@ public class VSizeIndexed implements IndexedMultivalue { inspector.visit("theBuffer", theBuffer); } - - public WritableSupplier> asWritableSupplier() - { - return new VSizeIndexedSupplier(this); - } - - public static class VSizeIndexedSupplier implements WritableSupplier> - { - final VSizeIndexed delegate; - - public VSizeIndexedSupplier(VSizeIndexed delegate) - { - this.delegate = delegate; - } - - @Override - public long getSerializedSize() - { - return delegate.getSerializedSize(); - } - - @Override - public void writeToChannel(WritableByteChannel channel) throws IOException - { - delegate.writeToChannel(channel); - } - - @Override - public IndexedMultivalue get() - { - return delegate; - } - } } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java index c9bee91cf82..1c499040bb0 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -20,20 +20,31 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; +import io.druid.common.utils.ByteUtils; +import io.druid.io.Channels; import io.druid.java.util.common.IAE; +import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.serde.MetaSerdeHelper; +import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; +import it.unimi.dsi.fastutil.ints.IntArrayList; +import it.unimi.dsi.fastutil.ints.IntList; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; -import java.util.List; /** */ -public class VSizeIndexedInts implements IndexedInts, Comparable +public class VSizeIndexedInts implements IndexedInts, Comparable, WritableSupplier { public static final byte VERSION = 0x0; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((VSizeIndexedInts x) -> VERSION) + .writeByte(x -> ByteUtils.checkedCast(x.numBytes)) + .writeInt(x -> x.buffer.remaining()); + public static VSizeIndexedInts fromArray(int[] array) { return fromArray(array, Ints.max(array)); @@ -41,23 +52,10 @@ public class VSizeIndexedInts implements IndexedInts, Comparable list, int maxValue) - { - int numBytes = getNumBytesForMax(maxValue); - - final ByteBuffer buffer = ByteBuffer.allocate((list.size() * numBytes)); - writeToBuffer(buffer, list, numBytes, maxValue); - - return buffer.array(); - } - - public static VSizeIndexedInts fromList(List list, int maxValue) + public static VSizeIndexedInts fromList(IntList list, int maxValue) { int numBytes = getNumBytesForMax(maxValue); @@ -67,11 +65,11 @@ public class VSizeIndexedInts implements IndexedInts, Comparable list, int numBytes, int maxValue) + private static void writeToBuffer(ByteBuffer buffer, IntList list, int numBytes, int maxValue) { - int i = 0; ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); - for (Integer val : list) { + for (int i = 0; i < list.size(); i++) { + int val = list.getInt(i); if (val < 0) { throw new IAE("integer values must be positive, got[%d], i[%d]", val, i); } @@ -81,7 +79,6 @@ public class VSizeIndexedInts implements IndexedInts, Comparable>> bitsToShift; } - public byte[] getBytesNoPadding() + public int getNumBytesNoPadding() { - int bytesToTake = buffer.remaining() - (4 - numBytes); - byte[] bytes = new byte[bytesToTake]; - buffer.asReadOnlyBuffer().get(bytes); - return bytes; + return buffer.remaining() - (Ints.BYTES - numBytes); + } + + public void writeBytesNoPaddingTo(HeapByteBufferWriteOutBytes out) + { + ByteBuffer toWrite = buffer.slice(); + toWrite.limit(toWrite.limit() - (Ints.BYTES - numBytes)); + out.write(toWrite); } @Override @@ -157,17 +157,23 @@ public class VSizeIndexedInts implements IndexedInts, Comparable asWritableSupplier() - { - return new VSizeIndexedIntsSupplier(this); - } - - public static class VSizeIndexedIntsSupplier implements WritableSupplier - { - final VSizeIndexedInts delegate; - - public VSizeIndexedIntsSupplier(VSizeIndexedInts delegate) - { - this.delegate = delegate; - } - - @Override - public long getSerializedSize() - { - return delegate.getSerializedSize(); - } - - @Override - public void writeToChannel(WritableByteChannel channel) throws IOException - { - delegate.writeToChannel(channel); - } - - @Override - public IndexedInts get() - { - return delegate; - } - } } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java index 624aa3d4e73..edbaf2f4f6c 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java @@ -19,16 +19,15 @@ package io.druid.segment.data; -import com.google.common.io.ByteStreams; -import com.google.common.io.CountingOutputStream; import com.google.common.primitives.Ints; -import io.druid.java.util.common.StringUtils; +import io.druid.common.utils.ByteUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; /** @@ -38,61 +37,61 @@ public class VSizeIndexedIntsWriter extends SingleValueIndexedIntsWriter { private static final byte VERSION = VSizeIndexedInts.VERSION; - private final IOPeon ioPeon; - private final String valueFileName; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((VSizeIndexedIntsWriter x) -> VERSION) + .writeByte(x -> ByteUtils.checkedCast(x.numBytes)) + .writeInt(x -> Ints.checkedCast(x.valuesOut.size())); + + private final SegmentWriteOutMedium segmentWriteOutMedium; private final int numBytes; - private CountingOutputStream valuesOut = null; private final ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); + private WriteOutBytes valuesOut = null; + private boolean bufPaddingWritten = false; - public VSizeIndexedIntsWriter( - final IOPeon ioPeon, - final String filenameBase, - final int maxValue - ) + public VSizeIndexedIntsWriter(final SegmentWriteOutMedium segmentWriteOutMedium, final int maxValue) { - this.ioPeon = ioPeon; - this.valueFileName = StringUtils.format("%s.values", filenameBase); + this.segmentWriteOutMedium = segmentWriteOutMedium; this.numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue); } @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFileName)); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override protected void addValue(int val) throws IOException { + if (bufPaddingWritten) { + throw new IllegalStateException("written out already"); + } helperBuffer.putInt(0, val); valuesOut.write(helperBuffer.array(), Ints.BYTES - numBytes, numBytes); } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - byte[] bufPadding = new byte[4 - numBytes]; - valuesOut.write(bufPadding); - valuesOut.close(); + writeBufPadding(); + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override - public long getSerializedSize() + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - return 2 + // version and numBytes - 4 + // dataLen - valuesOut.getCount(); + writeBufPadding(); + metaSerdeHelper.writeTo(channel, this); + valuesOut.writeTo(channel); } - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private void writeBufPadding() throws IOException { - long numBytesWritten = valuesOut.getCount(); - channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes})); - channel.write(ByteBuffer.wrap(Ints.toByteArray((int) numBytesWritten))); - try (final ReadableByteChannel from = Channels.newChannel(ioPeon.makeInputStream(valueFileName))) { - ByteStreams.copy(from, channel); + if (!bufPaddingWritten) { + byte[] bufPadding = new byte[Ints.BYTES - numBytes]; + valuesOut.write(bufPadding); + bufPaddingWritten = true; } } } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java index 732b3722ad5..7f4f1d4e1f2 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java @@ -19,152 +19,145 @@ package io.druid.segment.data; -import com.google.common.base.Function; import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; -import com.google.common.io.ByteStreams; -import com.google.common.io.Closeables; -import com.google.common.io.CountingOutputStream; -import com.google.common.io.InputSupplier; import com.google.common.primitives.Ints; -import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.serde.MetaSerdeHelper; +import it.unimi.dsi.fastutil.ints.IntList; -import java.io.Closeable; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; -import java.util.Arrays; -import java.util.List; /** * Streams arrays of objects out in the binary format described by VSizeIndexed */ -public class VSizeIndexedWriter extends MultiValueIndexedIntsWriter implements Closeable +public class VSizeIndexedWriter extends MultiValueIndexedIntsWriter { private static final byte VERSION = 0x1; - private static final byte[] EMPTY_ARRAY = new byte[]{}; + + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((VSizeIndexedWriter x) -> VERSION) + .writeByte(x -> VSizeIndexedInts.getNumBytesForMax(x.maxId)) + .writeInt(x -> Ints.checkedCast(x.headerOut.size() + x.valuesOut.size() + Integer.BYTES)) + .writeInt(x -> x.numWritten); + + private enum WriteInt + { + ONE_BYTE { + @Override + void write(WriteOutBytes out, int v) throws IOException + { + out.write(v); + } + }, + TWO_BYTES { + @Override + void write(WriteOutBytes out, int v) throws IOException + { + out.write(v >> 8); + out.write(v); + } + }, + THREE_BYTES { + @Override + void write(WriteOutBytes out, int v) throws IOException + { + out.write(v >> 16); + out.write(v >> 8); + out.write(v); + } + }, + FOUR_BYTES { + @Override + void write(WriteOutBytes out, int v) throws IOException + { + out.writeInt(v); + } + }; + + abstract void write(WriteOutBytes out, int v) throws IOException; + } private final int maxId; + private final WriteInt writeInt; - private CountingOutputStream headerOut = null; - private CountingOutputStream valuesOut = null; - int numWritten = 0; - private final IOPeon ioPeon; - private final String metaFileName; - private final String headerFileName; - private final String valuesFileName; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private WriteOutBytes headerOut = null; + private WriteOutBytes valuesOut = null; + private int numWritten = 0; + private boolean numBytesForMaxWritten = false; - public VSizeIndexedWriter( - IOPeon ioPeon, - String filenameBase, - int maxId - ) + public VSizeIndexedWriter(SegmentWriteOutMedium segmentWriteOutMedium, int maxId) { - this.ioPeon = ioPeon; - this.metaFileName = StringUtils.format("%s.meta", filenameBase); - this.headerFileName = StringUtils.format("%s.header", filenameBase); - this.valuesFileName = StringUtils.format("%s.values", filenameBase); + this.segmentWriteOutMedium = segmentWriteOutMedium; this.maxId = maxId; + this.writeInt = WriteInt.values()[VSizeIndexedInts.getNumBytesForMax(maxId) - 1]; } @Override public void open() throws IOException { - headerOut = new CountingOutputStream(ioPeon.makeOutputStream(headerFileName)); - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valuesFileName)); + headerOut = segmentWriteOutMedium.makeWriteOutBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override - protected void addValues(List val) throws IOException + protected void addValues(IntList ints) throws IOException { - write(val); - } - - public void write(List ints) throws IOException - { - byte[] bytesToWrite = ints == null ? EMPTY_ARRAY : VSizeIndexedInts.getBytesNoPaddingFromList(ints, maxId); - - valuesOut.write(bytesToWrite); - - headerOut.write(Ints.toByteArray((int) valuesOut.getCount())); - + if (numBytesForMaxWritten) { + throw new IllegalStateException("written out already"); + } + if (ints != null) { + for (int i = 0; i < ints.size(); i++) { + int value = ints.getInt(i); + Preconditions.checkState(value >= 0 && value <= maxId); + writeInt.write(valuesOut, value); + } + } + headerOut.writeInt(Ints.checkedCast(valuesOut.size())); ++numWritten; } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - final byte numBytesForMax = VSizeIndexedInts.getNumBytesForMax(maxId); + writeNumBytesForMax(); + return metaSerdeHelper.size(this) + headerOut.size() + valuesOut.size(); + } - valuesOut.write(new byte[4 - numBytesForMax]); + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + writeNumBytesForMax(); - Closeables.close(headerOut, false); - Closeables.close(valuesOut, false); - - final long numBytesWritten = headerOut.getCount() + valuesOut.getCount(); + final long numBytesWritten = headerOut.size() + valuesOut.size(); Preconditions.checkState( - headerOut.getCount() == (numWritten * 4), + headerOut.size() == (numWritten * 4), "numWritten[%s] number of rows should have [%s] bytes written to headerOut, had[%s]", numWritten, numWritten * 4, - headerOut.getCount() + headerOut.size() ); Preconditions.checkState( - numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten + numBytesWritten < Integer.MAX_VALUE - Integer.BYTES, + "Wrote[%s] bytes, which is too many.", + numBytesWritten ); - try (OutputStream metaOut = ioPeon.makeOutputStream(metaFileName)) { - metaOut.write(new byte[]{VERSION, numBytesForMax}); - metaOut.write(Ints.toByteArray((int) numBytesWritten + 4)); - metaOut.write(Ints.toByteArray(numWritten)); - } + metaSerdeHelper.writeTo(channel, this); + headerOut.writeTo(channel); + valuesOut.writeTo(channel); } - public InputSupplier combineStreams() + private void writeNumBytesForMax() throws IOException { - return ByteStreams.join( - Iterables.transform( - Arrays.asList(metaFileName, headerFileName, valuesFileName), - new Function>() - { - @Override - public InputSupplier apply(final String input) - { - return new InputSupplier() - { - @Override - public InputStream getInput() throws IOException - { - return ioPeon.makeInputStream(input); - } - }; - } - } - ) - ); - } - - @Override - public long getSerializedSize() - { - return 1 + // version - 1 + // numBytes - 4 + // numBytesWritten - 4 + // numElements - headerOut.getCount() + - valuesOut.getCount(); - } - - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - try (final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput())) { - ByteStreams.copy(from, channel); + if (!numBytesForMaxWritten) { + final byte numBytesForMax = VSizeIndexedInts.getNumBytesForMax(maxId); + valuesOut.write(new byte[4 - numBytesForMax]); + numBytesForMaxWritten = true; } } } diff --git a/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java b/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java index 585f949724c..4ca906b398a 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeLongSerde.java @@ -191,6 +191,7 @@ public class VSizeLongSerde ByteBuffer buffer; byte curByte = 0; int count = 0; + private boolean closed = false; public Size1Ser(OutputStream output) { @@ -222,6 +223,9 @@ public class VSizeLongSerde @Override public void close() throws IOException { + if (closed) { + return; + } buffer.put((byte) (curByte << (8 - count))); if (output != null) { output.write(buffer.array()); @@ -230,6 +234,7 @@ public class VSizeLongSerde } else { buffer.putInt(0); } + closed = true; } } @@ -239,6 +244,7 @@ public class VSizeLongSerde ByteBuffer buffer; byte curByte = 0; int count = 0; + private boolean closed = false; public Size2Ser(OutputStream output) { @@ -270,6 +276,9 @@ public class VSizeLongSerde @Override public void close() throws IOException { + if (closed) { + return; + } buffer.put((byte) (curByte << (8 - count))); if (output != null) { output.write(buffer.array()); @@ -278,6 +287,7 @@ public class VSizeLongSerde } else { buffer.putInt(0); } + closed = true; } } @@ -289,6 +299,7 @@ public class VSizeLongSerde int numBytes; byte curByte = 0; boolean first = true; + private boolean closed = false; public Mult4Ser(OutputStream output, int numBytes) { @@ -329,6 +340,9 @@ public class VSizeLongSerde @Override public void close() throws IOException { + if (closed) { + return; + } if (!first) { buffer.put((byte) (curByte << 4)); } @@ -339,6 +353,7 @@ public class VSizeLongSerde } else { buffer.putInt(0); } + closed = true; } } @@ -347,6 +362,7 @@ public class VSizeLongSerde OutputStream output; ByteBuffer buffer; int numBytes; + private boolean closed = false; public Mult8Ser(OutputStream output, int numBytes) { @@ -377,12 +393,16 @@ public class VSizeLongSerde @Override public void close() throws IOException { + if (closed) { + return; + } if (output != null) { output.write(EMPTY); output.flush(); } else { buffer.putInt(0); } + closed = true; } } diff --git a/processing/src/main/java/io/druid/segment/data/WritableSupplier.java b/processing/src/main/java/io/druid/segment/data/WritableSupplier.java index 5e704aa37ed..49bffd57924 100644 --- a/processing/src/main/java/io/druid/segment/data/WritableSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/WritableSupplier.java @@ -20,12 +20,8 @@ package io.druid.segment.data; import com.google.common.base.Supplier; +import io.druid.segment.serde.Serializer; -import java.io.IOException; -import java.nio.channels.WritableByteChannel; - -public interface WritableSupplier extends Supplier +public interface WritableSupplier extends Supplier, Serializer { - long getSerializedSize(); - void writeToChannel(WritableByteChannel channel) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java index fe75295f71b..4c7d7b1fd86 100644 --- a/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ColumnPartSerde.java @@ -21,13 +21,10 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; -import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.WritableByteChannel; /** */ @@ -45,13 +42,6 @@ public interface ColumnPartSerde Deserializer getDeserializer(); - interface Serializer - { - long numBytes(); - - void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; - } - interface Deserializer { void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig); diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java index 2b38f5ae70d..432e1bf7a39 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnPartSerde.java @@ -21,14 +21,11 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; -import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.WritableByteChannel; /** */ @@ -103,23 +100,7 @@ public class ComplexColumnPartSerde implements ColumnPartSerde public ComplexColumnPartSerde build() { - return new ComplexColumnPartSerde( - typeName, - new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - delegate.writeToChannel(channel, smoosher); - } - } - ); + return new ComplexColumnPartSerde(typeName, delegate); } } } diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java b/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java index a7687849100..97798eaeddc 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java @@ -22,9 +22,9 @@ package io.druid.segment.serde; import io.druid.guice.annotations.PublicApi; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.data.GenericIndexedWriter; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import java.io.IOException; @@ -32,38 +32,29 @@ import java.nio.channels.WritableByteChannel; public class ComplexColumnSerializer implements GenericColumnSerializer { - private final IOPeon ioPeon; + @PublicApi + public static ComplexColumnSerializer create(SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy) + { + return new ComplexColumnSerializer(segmentWriteOutMedium, filenameBase, strategy); + } + + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ObjectStrategy strategy; private GenericIndexedWriter writer; - public ComplexColumnSerializer( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) + + private ComplexColumnSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy) { - this.ioPeon = ioPeon; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.strategy = strategy; } - @PublicApi - public static ComplexColumnSerializer create( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) - { - return new ComplexColumnSerializer(ioPeon, filenameBase, strategy); - } - @SuppressWarnings(value = "unchecked") @Override public void open() throws IOException { - writer = new GenericIndexedWriter( - ioPeon, StringUtils.format("%s.complex_column", filenameBase), strategy - ); + writer = new GenericIndexedWriter(segmentWriteOutMedium, StringUtils.format("%s.complex_column", filenameBase), strategy); writer.open(); } @@ -75,28 +66,20 @@ public class ComplexColumnSerializer implements GenericColumnSerializer } @Override - public void close() throws IOException - { - writer.close(); - } - - @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { return writer.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - writeToChannelVersionOne(channel); + writeToVersionOne(channel); } - private void writeToChannelVersionOne(WritableByteChannel channel) throws IOException + private void writeToVersionOne(WritableByteChannel channel) throws IOException { - writer.writeToChannel( - channel, - null - ); //null for the FileSmoosher means that we default to "version 1" of GenericIndexed. + //null for the FileSmoosher means that we default to "version 1" of GenericIndexed. + writer.writeTo(channel, null); } } diff --git a/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java b/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java index e0b2aaa662d..dc3c6647f3a 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java @@ -21,9 +21,9 @@ package io.druid.segment.serde; import com.google.common.base.Function; import io.druid.guice.annotations.ExtensionPoint; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import java.nio.ByteBuffer; @@ -108,12 +108,10 @@ public abstract class ComplexMetricSerde * For large column (i.e columns greater than Integer.MAX) use * (@link LargeColumnSupportedComplexColumnSerializer) * - * @param peon IOPeon - * @param column name of the column * @return an instance of GenericColumnSerializer used for serialization. */ - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return ComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return ComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } } diff --git a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java index 319fd550b2f..c77078efb41 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -25,9 +25,9 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.spatial.ImmutableRTree; +import io.druid.io.Channels; import io.druid.java.util.common.IAE; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.segment.CompressedVSizeIndexedSupplier; import io.druid.segment.CompressedVSizeIndexedV3Supplier; import io.druid.segment.column.ColumnBuilder; @@ -211,7 +211,7 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde new Serializer() { @Override - public long numBytes() + public long getSerializedSize() throws IOException { long size = 1 + // version (version.compareTo(VERSION.COMPRESSED) >= 0 @@ -233,23 +233,23 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde } @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{version.asByte()})); + Channels.writeFully(channel, ByteBuffer.wrap(new byte[]{version.asByte()})); if (version.compareTo(VERSION.COMPRESSED) >= 0) { channel.write(ByteBuffer.wrap(Ints.toByteArray(flags))); } if (dictionaryWriter != null) { - dictionaryWriter.writeToChannel(channel, smoosher); + dictionaryWriter.writeTo(channel, smoosher); } if (valueWriter != null) { - valueWriter.writeToChannel(channel, smoosher); + valueWriter.writeTo(channel, smoosher); } if (bitmapIndexWriter != null) { - bitmapIndexWriter.writeToChannel(channel, smoosher); + bitmapIndexWriter.writeTo(channel, smoosher); } if (spatialIndexWriter != null) { - spatialIndexWriter.writeToChannel(channel); + spatialIndexWriter.writeTo(channel, smoosher); } } } @@ -295,10 +295,10 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde final WritableSupplier> rMultiValuedColumn; if (hasMultipleValues) { - rMultiValuedColumn = readMultiValuedColumn(rVersion, buffer, rFlags, builder.getFileMapper()); + rMultiValuedColumn = readMultiValuedColumn(rVersion, buffer, rFlags); rSingleValuedColumn = null; } else { - rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer, builder.getFileMapper()); + rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer); rMultiValuedColumn = null; } @@ -324,42 +324,39 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde ImmutableRTree rSpatialIndex = null; if (buffer.hasRemaining()) { rSpatialIndex = ByteBufferSerializer.read( - buffer, new ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()) + buffer, + new ImmutableRTreeObjectStrategy(bitmapSerdeFactory.getBitmapFactory()) ); builder.setSpatialIndex(new SpatialIndexColumnPartSupplier(rSpatialIndex)); } } - private WritableSupplier readSingleValuedColumn( - VERSION version, - ByteBuffer buffer, - SmooshedFileMapper fileMapper - ) + private WritableSupplier readSingleValuedColumn(VERSION version, ByteBuffer buffer) { switch (version) { case UNCOMPRESSED_SINGLE_VALUE: - return VSizeIndexedInts.readFromByteBuffer(buffer).asWritableSupplier(); + return VSizeIndexedInts.readFromByteBuffer(buffer); case COMPRESSED: - return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder, fileMapper); + return CompressedVSizeIntsIndexedSupplier.fromByteBuffer(buffer, byteOrder); default: throw new IAE("Unsupported single-value version[%s]", version); } } private WritableSupplier> readMultiValuedColumn( - VERSION version, ByteBuffer buffer, int flags, SmooshedFileMapper fileMapper + VERSION version, ByteBuffer buffer, int flags ) { switch (version) { case UNCOMPRESSED_MULTI_VALUE: { - return VSizeIndexed.readFromByteBuffer(buffer).asWritableSupplier(); + return VSizeIndexed.readFromByteBuffer(buffer); } case COMPRESSED: { if (Feature.MULTI_VALUE.isSet(flags)) { - return CompressedVSizeIndexedSupplier.fromByteBuffer(buffer, byteOrder, fileMapper); + return CompressedVSizeIndexedSupplier.fromByteBuffer(buffer, byteOrder); } else if (Feature.MULTI_VALUE_V3.isSet(flags)) { - return CompressedVSizeIndexedV3Supplier.fromByteBuffer(buffer, byteOrder, fileMapper); + return CompressedVSizeIndexedV3Supplier.fromByteBuffer(buffer, byteOrder); } else { throw new IAE("Unrecognized multi-value flag[%d]", flags); } diff --git a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java index 55e93139219..b25388db426 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java @@ -19,25 +19,21 @@ package io.druid.segment.serde; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Supplier; -import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.DoubleColumnSerializer; +import io.druid.segment.column.ColumnBuilder; +import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedDoublesIndexedSupplier; import io.druid.segment.data.IndexedDoubles; -import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.WritableByteChannel; public class DoubleGenericColumnPartSerde implements ColumnPartSerde { - private final ByteOrder byteOrder; - private Serializer serialize; - @JsonCreator public static DoubleGenericColumnPartSerde getDoubleGenericColumnPartSerde( @JsonProperty("byteOrder") ByteOrder byteOrder @@ -46,41 +42,21 @@ public class DoubleGenericColumnPartSerde implements ColumnPartSerde return new DoubleGenericColumnPartSerde(byteOrder, null); } + private final ByteOrder byteOrder; + private final Serializer serializer; + + private DoubleGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) + { + this.byteOrder = byteOrder; + this.serializer = serializer; + } + @JsonProperty public ByteOrder getByteOrder() { return byteOrder; } - - public DoubleGenericColumnPartSerde(ByteOrder byteOrder, Serializer serialize) - { - this.byteOrder = byteOrder; - this.serialize = serialize; - } - - @Override - public Serializer getSerializer() - { - return serialize; - } - - @Override - public Deserializer getDeserializer() - { - return (buffer, builder, columnConfig) -> { - final Supplier column = CompressedDoublesIndexedSupplier.fromByteBuffer( - buffer, - byteOrder, - builder.getFileMapper() - ); - builder.setType(ValueType.DOUBLE) - .setHasMultipleValues(false) - .setGenericColumn(new DoubleGenericColumnSupplier(column)); - - }; - } - public static SerializerBuilder serializerBuilder() { return new SerializerBuilder(); @@ -105,23 +81,33 @@ public class DoubleGenericColumnPartSerde implements ColumnPartSerde public DoubleGenericColumnPartSerde build() { - return new DoubleGenericColumnPartSerde( - byteOrder, - new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher fileSmoosher) throws IOException - { - delegate.writeToChannel(channel, fileSmoosher); - } - } - ); + return new DoubleGenericColumnPartSerde(byteOrder, delegate); } } + + @Override + public Serializer getSerializer() + { + return serializer; + } + + @Override + public Deserializer getDeserializer() + { + return new Deserializer() + { + @Override + public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnConfig) + { + final Supplier column = CompressedDoublesIndexedSupplier.fromByteBuffer( + buffer, + byteOrder + ); + builder.setType(ValueType.DOUBLE) + .setHasMultipleValues(false) + .setGenericColumn(new DoubleGenericColumnSupplier(column)); + + } + }; + } } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java index 68d35721293..bd26cfb9a00 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnPartSerde.java @@ -21,18 +21,14 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - -import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.FloatColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedFloatsIndexedSupplier; -import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.WritableByteChannel; /** */ @@ -47,7 +43,7 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde } private final ByteOrder byteOrder; - private Serializer serializer; + private final Serializer serializer; private FloatGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) { @@ -85,23 +81,7 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde public FloatGenericColumnPartSerde build() { - return new FloatGenericColumnPartSerde( - byteOrder, - new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher fileSmoosher) throws IOException - { - delegate.writeToChannel(channel, fileSmoosher); - } - } - ); + return new FloatGenericColumnPartSerde(byteOrder, delegate); } } @@ -121,8 +101,7 @@ public class FloatGenericColumnPartSerde implements ColumnPartSerde { final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer( buffer, - byteOrder, - builder.getFileMapper() + byteOrder ); builder.setType(ValueType.FLOAT) .setHasMultipleValues(false) diff --git a/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java b/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java index f3c12d8a919..6486f7eeaa0 100644 --- a/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java @@ -22,9 +22,9 @@ package io.druid.segment.serde; import io.druid.guice.annotations.PublicApi; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.data.GenericIndexedWriter; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import java.io.IOException; @@ -32,59 +32,64 @@ import java.nio.channels.WritableByteChannel; public class LargeColumnSupportedComplexColumnSerializer implements GenericColumnSerializer { + @PublicApi + public static LargeColumnSupportedComplexColumnSerializer create( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + ObjectStrategy strategy + ) + { + return new LargeColumnSupportedComplexColumnSerializer(segmentWriteOutMedium, filenameBase, strategy); + } - private final IOPeon ioPeon; + public static LargeColumnSupportedComplexColumnSerializer createWithColumnSize( + SegmentWriteOutMedium segmentWriteOutMedium, + String filenameBase, + ObjectStrategy strategy, + int columnSize + ) + { + return new LargeColumnSupportedComplexColumnSerializer(segmentWriteOutMedium, filenameBase, strategy, columnSize); + } + + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ObjectStrategy strategy; private final int columnSize; private GenericIndexedWriter writer; - public LargeColumnSupportedComplexColumnSerializer( - IOPeon ioPeon, + + private LargeColumnSupportedComplexColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy ) { - this(ioPeon, filenameBase, strategy, Integer.MAX_VALUE); + this(segmentWriteOutMedium, filenameBase, strategy, Integer.MAX_VALUE); } - public LargeColumnSupportedComplexColumnSerializer( - IOPeon ioPeon, + + private LargeColumnSupportedComplexColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy, int columnSize ) { - this.ioPeon = ioPeon; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.strategy = strategy; this.columnSize = columnSize; } - @PublicApi - public static LargeColumnSupportedComplexColumnSerializer create( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) - { - return new LargeColumnSupportedComplexColumnSerializer(ioPeon, filenameBase, strategy); - } - - public static LargeColumnSupportedComplexColumnSerializer createWithColumnSize( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy, - int columnSize - ) - { - return new LargeColumnSupportedComplexColumnSerializer(ioPeon, filenameBase, strategy, columnSize); - } - @SuppressWarnings(value = "unchecked") @Override public void open() throws IOException { writer = new GenericIndexedWriter( - ioPeon, StringUtils.format("%s.complex_column", filenameBase), strategy, columnSize); + segmentWriteOutMedium, + StringUtils.format("%s.complex_column", filenameBase), + strategy, + columnSize + ); writer.open(); } @@ -96,21 +101,15 @@ public class LargeColumnSupportedComplexColumnSerializer implements GenericColum } @Override - public void close() throws IOException - { - writer.close(); - } - - @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { return writer.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - writer.writeToChannel(channel, smoosher); + writer.writeTo(channel, smoosher); } } diff --git a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index e88d131e356..758bb07fa30 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -21,18 +21,14 @@ package io.druid.segment.serde; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - -import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.segment.LongColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnConfig; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressedLongsIndexedSupplier; -import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.WritableByteChannel; /** */ @@ -47,7 +43,7 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde } private final ByteOrder byteOrder; - private Serializer serializer; + private final Serializer serializer; private LongGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) { @@ -85,23 +81,7 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde public LongGenericColumnPartSerde build() { - return new LongGenericColumnPartSerde( - byteOrder, - new Serializer() - { - @Override - public long numBytes() - { - return delegate.getSerializedSize(); - } - - @Override - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException - { - delegate.writeToChannel(channel, smoosher); - } - } - ); + return new LongGenericColumnPartSerde(byteOrder, delegate); } } @@ -121,8 +101,7 @@ public class LongGenericColumnPartSerde implements ColumnPartSerde { final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer( buffer, - byteOrder, - builder.getFileMapper() + byteOrder ); builder.setType(ValueType.LONG) .setHasMultipleValues(false) diff --git a/processing/src/main/java/io/druid/segment/serde/MetaSerdeHelper.java b/processing/src/main/java/io/druid/segment/serde/MetaSerdeHelper.java new file mode 100644 index 00000000000..0e8c6987d38 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/MetaSerdeHelper.java @@ -0,0 +1,158 @@ +/* + * 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.serde; + +import io.druid.io.Channels; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; +import java.util.function.Predicate; + +public final class MetaSerdeHelper +{ + public static MetaSerdeHelper firstWriteByte(ByteFieldWriter fieldWriter) + { + return new MetaSerdeHelper().writeByte(fieldWriter); + } + + private final List> fieldWriters = new ArrayList<>(); + + private MetaSerdeHelper() {} + + public MetaSerdeHelper writeInt(IntFieldWriter fieldWriter) + { + return writeSomething(fieldWriter); + } + + public MetaSerdeHelper writeByte(ByteFieldWriter fieldWriter) + { + return writeSomething(fieldWriter); + } + + public MetaSerdeHelper maybeWriteByte(Predicate condition, ByteFieldWriter fieldWriter) + { + return writeSomething( + new FieldWriter() + { + @Override + public void writeTo(ByteBuffer buffer, T x) throws IOException + { + if (condition.test(x)) { + buffer.put(fieldWriter.getField(x)); + } + } + + @Override + public int size(T x) + { + return condition.test(x) ? Byte.BYTES : 0; + } + } + ); + } + + public MetaSerdeHelper writeByteArray(Function getByteArray) + { + return writeSomething( + new FieldWriter() + { + @Override + public void writeTo(ByteBuffer buffer, T x) throws IOException + { + buffer.put(getByteArray.apply(x)); + } + + @Override + public int size(T x) + { + return getByteArray.apply(x).length; + } + } + ); + } + + public MetaSerdeHelper writeSomething(FieldWriter fieldWriter) + { + fieldWriters.add(fieldWriter); + return this; + } + + public void writeTo(WritableByteChannel channel, T x) throws IOException + { + ByteBuffer meta = ByteBuffer.allocate(size(x)); + for (FieldWriter w : fieldWriters) { + w.writeTo(meta, x); + } + meta.flip(); + Channels.writeFully(channel, meta); + } + + public int size(T x) + { + return fieldWriters.stream().mapToInt(w -> w.size(x)).sum(); + } + + public interface FieldWriter + { + void writeTo(ByteBuffer buffer, T x) throws IOException; + + int size(T x); + } + + @FunctionalInterface + public interface IntFieldWriter extends FieldWriter + { + int getField(T x) throws IOException; + + @Override + default void writeTo(ByteBuffer buffer, T x) throws IOException + { + buffer.putInt(getField(x)); + } + + @Override + default int size(T x) + { + return Integer.BYTES; + } + } + + @FunctionalInterface + public interface ByteFieldWriter extends FieldWriter + { + byte getField(T x) throws IOException; + + @Override + default void writeTo(ByteBuffer buffer, T x) throws IOException + { + buffer.put(getField(x)); + } + + @Override + default int size(T x) + { + return Byte.BYTES; + } + } +} diff --git a/processing/src/main/java/io/druid/segment/serde/Serializer.java b/processing/src/main/java/io/druid/segment/serde/Serializer.java new file mode 100644 index 00000000000..5c20c11b502 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/Serializer.java @@ -0,0 +1,43 @@ +/* + * 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.serde; + +import io.druid.java.util.common.io.smoosh.FileSmoosher; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +/** + * Main interface for "serializeable something" in Druid segment serialization. + */ +public interface Serializer +{ + /** + * Returns the number of bytes, that this Serializer will write to the output _channel_ (not smoosher) on a {@link + * #writeTo} call. + */ + long getSerializedSize() throws IOException; + + /** + * Writes serialized form of this object to the given channel. If parallel data streams are needed, they could be + * created with the provided smoosher. + */ + void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/writeout/ByteBufferWriteOutBytes.java b/processing/src/main/java/io/druid/segment/writeout/ByteBufferWriteOutBytes.java new file mode 100644 index 00000000000..263c74679da --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/ByteBufferWriteOutBytes.java @@ -0,0 +1,265 @@ +/* + * 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.writeout; + +import com.google.common.base.Preconditions; +import com.google.common.io.ByteSource; +import com.google.common.primitives.Ints; +import io.druid.io.ByteBufferInputStream; +import io.druid.io.Channels; +import io.druid.java.util.common.IAE; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.ArrayList; +import java.util.function.Function; +import java.util.stream.Collectors; + +public abstract class ByteBufferWriteOutBytes extends WriteOutBytes +{ + /** + * There is no reason why 64K is chosen. Other power of 2 between 4K and 64K could be more reasonable. + */ + static final int BUFFER_SIZE = 64 * 1024; + + final ArrayList buffers = new ArrayList<>(); + int headBufferIndex; + ByteBuffer headBuffer; + long size; + long capacity; + + ByteBufferWriteOutBytes() + { + size = 0; + headBufferIndex = 0; + headBuffer = allocateBuffer(); + buffers.add(headBuffer); + capacity = BUFFER_SIZE; + } + + @Override + public long size() + { + return size; + } + + protected abstract ByteBuffer allocateBuffer(); + + private void ensureCapacity(int len) + { + long remaining = capacity - size; + for (long toAllocate = len - remaining; toAllocate >= 0; toAllocate -= BUFFER_SIZE) { + buffers.add(allocateBuffer()); + capacity += BUFFER_SIZE; + } + if (headBuffer.remaining() == 0) { + nextHead(); + } + } + + private void nextHead() + { + headBufferIndex++; + headBuffer = buffers.get(headBufferIndex); + } + + @Override + public void write(int b) + { + checkOpen(); + if (headBuffer.remaining() == 0) { + ensureCapacity(1); + } + headBuffer.put((byte) b); + size += 1; + } + + @Override + public void writeInt(int v) + { + checkOpen(); + if (headBuffer.remaining() >= Ints.BYTES) { + headBuffer.putInt(v); + size += Ints.BYTES; + } else { + ensureCapacity(Ints.BYTES); + if (headBuffer.remaining() >= Ints.BYTES) { + headBuffer.putInt(v); + size += Ints.BYTES; + } else { + write(v >> 24); + write(v >> 16); + write(v >> 8); + write(v); + } + } + } + + @Override + public void write(byte[] b) throws IOException + { + write0(b, 0, b.length); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + Preconditions.checkPositionIndexes(off, off + len, b.length); + write0(b, off, len); + } + + private void write0(byte[] b, int off, int len) + { + checkOpen(); + if (headBuffer.remaining() < len) { + ensureCapacity(len); + } + int headRemaining = headBuffer.remaining(); + if (len <= headRemaining) { + headBuffer.put(b, off, len); + } else { + headBuffer.put(b, off, headRemaining); + int bytesLeft = len - headRemaining; + off += headRemaining; + for (; bytesLeft > 0; bytesLeft -= BUFFER_SIZE, off += BUFFER_SIZE) { + nextHead(); + headBuffer.put(b, off, Math.min(BUFFER_SIZE, bytesLeft)); + } + } + size += len; + } + + @Override + public int write(ByteBuffer src) + { + checkOpen(); + int len = src.remaining(); + if (headBuffer.remaining() < len) { + ensureCapacity(len); + } + int headRemaining = headBuffer.remaining(); + src.limit(src.position() + Math.min(headRemaining, len)); + headBuffer.put(src); + for (int bytesLeft = len - headRemaining; bytesLeft > 0; bytesLeft -= BUFFER_SIZE) { + nextHead(); + src.limit(src.position() + Math.min(BUFFER_SIZE, bytesLeft)); + headBuffer.put(src); + } + size += len; + return len; + } + + @Override + public void writeTo(WritableByteChannel channel) throws IOException + { + checkOpen(); + for (int i = 0; i <= headBufferIndex; i++) { + ByteBuffer buffer = buffers.get(i); + buffer.flip(); + Channels.writeFully(channel, buffer); + // switch back to the initial state + buffer.limit(buffer.capacity()); + } + } + + /** + * Takes all bytes that are written to this WriteOutBytes so far and writes them into the given ByteBuffer. This method + * changes the position of the out buffer by the {@link #size()} of this WriteOutBytes. + * + * @throws java.nio.BufferOverflowException if the {@link ByteBuffer#remaining()} capacity of the given buffer is + * smaller than the size of this WriteOutBytes + */ + public void writeTo(ByteBuffer out) + { + checkOpen(); + for (int i = 0; i <= headBufferIndex; i++) { + ByteBuffer buffer = buffers.get(i); + buffer.flip(); + out.put(buffer); + // switch back to the initial state + buffer.limit(buffer.capacity()); + } + } + + @Override + public void readFully(long pos, ByteBuffer buffer) + { + checkOpen(); + if (pos < 0 || pos > size) { + throw new IAE("pos %d out of range [%d, %d]", pos, 0, size); + } + int ourBufferIndex = Ints.checkedCast(pos / BUFFER_SIZE); + int ourBufferOffset = Ints.checkedCast(pos % BUFFER_SIZE); + for (int bytesLeft = buffer.remaining(); bytesLeft > 0;) { + int bytesToWrite = Math.min(BUFFER_SIZE - ourBufferOffset, bytesLeft); + ByteBuffer ourBuffer = buffers.get(ourBufferIndex); + int ourBufferPosition = ourBuffer.position(); + if (bytesToWrite > ourBufferPosition - ourBufferOffset) { + throw new BufferUnderflowException(); + } + try { + ourBuffer.position(ourBufferOffset); + ourBuffer.limit(ourBufferOffset + bytesToWrite); + buffer.put(ourBuffer); + } + finally { + // switch back to the initial state + ourBuffer.limit(ourBuffer.capacity()); + ourBuffer.position(ourBufferPosition); + } + ourBufferIndex++; + ourBufferOffset = 0; + bytesLeft -= bytesToWrite; + } + } + + @Override + public InputStream asInputStream() throws IOException + { + checkOpen(); + Function byteBufferToByteSource = buf -> new ByteSource() + { + @Override + public InputStream openStream() + { + ByteBuffer inputBuf = buf.duplicate(); + inputBuf.flip(); + return new ByteBufferInputStream(inputBuf); + } + }; + return ByteSource.concat(buffers.stream().map(byteBufferToByteSource).collect(Collectors.toList())).openStream(); + } + + @Override + public boolean isOpen() + { + return true; + } + + private void checkOpen() + { + if (!isOpen()) { + throw new IllegalStateException(); + } + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/DirectByteBufferWriteOutBytes.java b/processing/src/main/java/io/druid/segment/writeout/DirectByteBufferWriteOutBytes.java new file mode 100644 index 00000000000..172c99afb78 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/DirectByteBufferWriteOutBytes.java @@ -0,0 +1,52 @@ +/* + * 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.writeout; + +import io.druid.java.util.common.ByteBufferUtils; + +import java.nio.ByteBuffer; + +final class DirectByteBufferWriteOutBytes extends ByteBufferWriteOutBytes +{ + private boolean open = true; + + @Override + protected ByteBuffer allocateBuffer() + { + return ByteBuffer.allocateDirect(BUFFER_SIZE); + } + + @Override + public boolean isOpen() + { + return open; + } + + void free() + { + open = false; + buffers.forEach(ByteBufferUtils::free); + buffers.clear(); + headBufferIndex = -1; + headBuffer = null; + size = 0; + capacity = 0; + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/FileWriteOutBytes.java b/processing/src/main/java/io/druid/segment/writeout/FileWriteOutBytes.java new file mode 100644 index 00000000000..9844afd9497 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/FileWriteOutBytes.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.writeout; + +import com.google.common.io.ByteStreams; +import io.druid.io.Channels; +import io.druid.java.util.common.IAE; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; + +final class FileWriteOutBytes extends WriteOutBytes +{ + private final File file; + private final FileChannel ch; + + /** Purposely big-endian, for {@link #writeInt(int)} implementation */ + private final ByteBuffer buffer = ByteBuffer.allocate(4096); // 4K page sized buffer + + FileWriteOutBytes(File file, FileChannel ch) + { + this.file = file; + this.ch = ch; + } + + private void flushIfNeeded(int bytesNeeded) throws IOException + { + if (buffer.remaining() < bytesNeeded) { + flush(); + } + } + + @Override + public void flush() throws IOException + { + buffer.flip(); + Channels.writeFully(ch, buffer); + buffer.clear(); + } + + @Override + public void write(int b) throws IOException + { + flushIfNeeded(1); + buffer.put((byte) b); + } + + @Override + public void writeInt(int v) throws IOException + { + flushIfNeeded(Integer.SIZE); + buffer.putInt(v); + } + + @Override + public int write(ByteBuffer src) throws IOException + { + int len = src.remaining(); + flushIfNeeded(len); + while (src.remaining() > buffer.capacity()) { + int srcLimit = src.limit(); + try { + src.limit(src.position() + buffer.capacity()); + buffer.put(src); + flush(); + } + finally { + // IOException may occur in flush(), reset src limit to the original + src.limit(srcLimit); + } + } + buffer.put(src); + return len; + } + + @Override + public void write(byte[] b, int off, int len) throws IOException + { + write(ByteBuffer.wrap(b, off, len)); + } + + @Override + public long size() throws IOException + { + flush(); + return ch.size(); + } + + @Override + public void writeTo(WritableByteChannel channel) throws IOException + { + flush(); + ch.position(0); + try { + ByteStreams.copy(ch, channel); + } + finally { + ch.position(ch.size()); + } + } + + @Override + public void readFully(long pos, ByteBuffer buffer) throws IOException + { + flush(); + if (pos < 0 || pos > ch.size()) { + throw new IAE("pos %d out of range [%d, %d]", pos, 0, ch.size()); + } + ch.read(buffer, pos); + if (buffer.remaining() > 0) { + throw new BufferUnderflowException(); + } + } + + @Override + public InputStream asInputStream() throws IOException + { + flush(); + return new FileInputStream(file); + } + + @Override + public boolean isOpen() + { + return ch.isOpen(); + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/HeapByteBufferWriteOutBytes.java b/processing/src/main/java/io/druid/segment/writeout/HeapByteBufferWriteOutBytes.java new file mode 100644 index 00000000000..4823d49375d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/HeapByteBufferWriteOutBytes.java @@ -0,0 +1,31 @@ +/* + * 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.writeout; + +import java.nio.ByteBuffer; + +public final class HeapByteBufferWriteOutBytes extends ByteBufferWriteOutBytes +{ + @Override + protected ByteBuffer allocateBuffer() + { + return ByteBuffer.allocate(BUFFER_SIZE); + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMedium.java b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMedium.java new file mode 100644 index 00000000000..4a8974d129e --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMedium.java @@ -0,0 +1,49 @@ +/* + * 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.writeout; + +import io.druid.java.util.common.io.Closer; + +import java.io.IOException; + +public final class OffHeapMemorySegmentWriteOutMedium implements SegmentWriteOutMedium +{ + private final Closer closer = Closer.create(); + + @Override + public WriteOutBytes makeWriteOutBytes() + { + DirectByteBufferWriteOutBytes writeOutBytes = new DirectByteBufferWriteOutBytes(); + closer.register(writeOutBytes::free); + return writeOutBytes; + } + + @Override + public Closer getCloser() + { + return closer; + } + + @Override + public void close() throws IOException + { + closer.close(); + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMediumFactory.java b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMediumFactory.java new file mode 100644 index 00000000000..d45c5b7908d --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMediumFactory.java @@ -0,0 +1,45 @@ +/* + * 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.writeout; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.File; +import java.io.IOException; + +public class OffHeapMemorySegmentWriteOutMediumFactory implements SegmentWriteOutMediumFactory +{ + private static final OffHeapMemorySegmentWriteOutMediumFactory INSTANCE = + new OffHeapMemorySegmentWriteOutMediumFactory(); + + @JsonCreator + public static OffHeapMemorySegmentWriteOutMediumFactory instance() + { + return INSTANCE; + } + + private OffHeapMemorySegmentWriteOutMediumFactory() {} + + @Override + public SegmentWriteOutMedium makeSegmentWriteOutMedium(File outDir) throws IOException + { + return new OffHeapMemorySegmentWriteOutMedium(); + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/OnHeapMemorySegmentWriteOutMedium.java b/processing/src/main/java/io/druid/segment/writeout/OnHeapMemorySegmentWriteOutMedium.java new file mode 100644 index 00000000000..ab83db62a6f --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/OnHeapMemorySegmentWriteOutMedium.java @@ -0,0 +1,49 @@ +/* + * 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.writeout; + +import com.google.common.annotations.VisibleForTesting; +import io.druid.java.util.common.io.Closer; + +import java.io.IOException; + +@VisibleForTesting +public final class OnHeapMemorySegmentWriteOutMedium implements SegmentWriteOutMedium +{ + private final Closer closer = Closer.create(); + + @Override + public WriteOutBytes makeWriteOutBytes() throws IOException + { + return new HeapByteBufferWriteOutBytes(); + } + + @Override + public Closer getCloser() + { + return closer; + } + + @Override + public void close() throws IOException + { + closer.close(); + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMedium.java b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMedium.java new file mode 100644 index 00000000000..4d16b7695bb --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMedium.java @@ -0,0 +1,47 @@ +/* + * 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.writeout; + +import io.druid.java.util.common.io.Closer; + +import java.io.Closeable; +import java.io.IOException; + +/** + * SegmentWriteOutMedium is an umbrella "resource disposer" for temporary buffers (in the form of {@link WriteOutBytes}, + * obtained by calling {@link #makeWriteOutBytes()} on the SegmentWriteOutMedium instance), that are used during new Druid + * segment creation, and other resources (see {@link #getCloser()}). + * + * When SegmentWriteOutMedium is closed, all child WriteOutBytes couldn't be used anymore. + */ +public interface SegmentWriteOutMedium extends Closeable +{ + /** + * Creates a new empty {@link WriteOutBytes}, attached to this SegmentWriteOutMedium. When this SegmentWriteOutMedium is + * closed, the returned WriteOutBytes couldn't be used anymore. + */ + WriteOutBytes makeWriteOutBytes() throws IOException; + + /** + * Returns a closer of this SegmentWriteOutMedium, which is closed in this SegmentWriteOutMedium's close() method. + * Could be used to "attach" some random resources to this SegmentWriteOutMedium, to be closed at the same time. + */ + Closer getCloser(); +} diff --git a/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumFactory.java b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumFactory.java new file mode 100644 index 00000000000..29c606ba23a --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumFactory.java @@ -0,0 +1,51 @@ +/* + * 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.writeout; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.ImmutableSet; + +import java.io.File; +import java.io.IOException; +import java.util.Set; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, defaultImpl = TmpFileSegmentWriteOutMediumFactory.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "tmpFile", value = TmpFileSegmentWriteOutMediumFactory.class), + @JsonSubTypes.Type(name = "offHeapMemory", value = OffHeapMemorySegmentWriteOutMediumFactory.class), +}) +public interface SegmentWriteOutMediumFactory +{ + static Set builtInFactories() + { + return ImmutableSet.of( + TmpFileSegmentWriteOutMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); + } + + /** + * Creates a new SegmentWriteOutMedium. If this type of SegmentWriteOutMedium needs to create some temprorary files, + * it creates a *subdirectory* in the given outDir, stores the files there, and removes the files and the subdirectory + * when closed. + */ + SegmentWriteOutMedium makeSegmentWriteOutMedium(File outDir) throws IOException; +} diff --git a/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumModule.java b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumModule.java new file mode 100644 index 00000000000..3f7a6fed204 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumModule.java @@ -0,0 +1,37 @@ +/* + * 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.writeout; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.druid.guice.JsonConfigProvider; + +public class SegmentWriteOutMediumModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind( + binder, + "druid.peon.defaultSegmentWriteOutMediumFactory", + SegmentWriteOutMediumFactory.class + ); + } +} diff --git a/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMedium.java b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMedium.java new file mode 100644 index 00000000000..b3ee365e732 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMedium.java @@ -0,0 +1,68 @@ +/* + * 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.writeout; + +import io.druid.java.util.common.io.Closer; +import org.apache.commons.io.FileUtils; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; + +public final class TmpFileSegmentWriteOutMedium implements SegmentWriteOutMedium +{ + private final File dir; + private final Closer closer = Closer.create(); + + TmpFileSegmentWriteOutMedium(File outDir) throws IOException + { + File tmpOutputFilesDir = new File(outDir, "tmpOutputFiles"); + FileUtils.forceMkdir(tmpOutputFilesDir); + closer.register(() -> FileUtils.deleteDirectory(tmpOutputFilesDir)); + this.dir = tmpOutputFilesDir; + } + + @Override + public WriteOutBytes makeWriteOutBytes() throws IOException + { + File file = File.createTempFile("filePeon", null, dir); + FileChannel ch = FileChannel.open( + file.toPath(), + StandardOpenOption.READ, + StandardOpenOption.WRITE + ); + closer.register(file::delete); + closer.register(ch); + return new FileWriteOutBytes(file, ch); + } + + @Override + public Closer getCloser() + { + return closer; + } + + @Override + public void close() throws IOException + { + closer.close(); + } +} diff --git a/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMediumFactory.java similarity index 54% rename from processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java rename to processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMediumFactory.java index afdd1fc99a3..7fce54f07ae 100644 --- a/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMediumFactory.java @@ -17,41 +17,28 @@ * under the License. */ -package io.druid.segment.data; +package io.druid.segment.writeout; -import java.nio.Buffer; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; +import com.fasterxml.jackson.annotation.JsonCreator; -public abstract class FixedSizeCompressedObjectStrategy extends CompressedObjectStrategy +import java.io.File; +import java.io.IOException; + +public final class TmpFileSegmentWriteOutMediumFactory implements SegmentWriteOutMediumFactory { - private final int sizePer; + private static final TmpFileSegmentWriteOutMediumFactory INSTANCE = new TmpFileSegmentWriteOutMediumFactory(); - protected FixedSizeCompressedObjectStrategy( - ByteOrder order, - BufferConverter converter, - CompressionStrategy compression, - int sizePer - ) + @JsonCreator + public static TmpFileSegmentWriteOutMediumFactory instance() { - super(order, converter, compression); - this.sizePer = sizePer; + return INSTANCE; } - public int getSize() - { - return sizePer; - } + private TmpFileSegmentWriteOutMediumFactory() {} @Override - protected ByteBuffer bufferFor(T val) + public SegmentWriteOutMedium makeSegmentWriteOutMedium(File outDir) throws IOException { - return ByteBuffer.allocate(converter.sizeOf(getSize())).order(order); - } - - @Override - protected void decompress(ByteBuffer buffer, int numBytes, ByteBuffer buf) - { - decompressor.decompress(buffer, numBytes, buf, converter.sizeOf(getSize())); + return new TmpFileSegmentWriteOutMedium(outDir); } } diff --git a/processing/src/main/java/io/druid/segment/writeout/WriteOutBytes.java b/processing/src/main/java/io/druid/segment/writeout/WriteOutBytes.java new file mode 100644 index 00000000000..312341ade44 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/writeout/WriteOutBytes.java @@ -0,0 +1,80 @@ +/* + * 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.writeout; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** + * Appendable byte sequence for temporary storage. Methods inherited from {@link OutputStream}, {@link + * WritableByteChannel} and {@link #writeInt(int)} append to the sequence. Methods {@link + * #writeTo(WritableByteChannel)} and {@link #asInputStream()} allow to write the sequence somewhere else. {@link + * #readFully} allows to access the sequence randomly. + * + * WriteOutBytes is a resource that is managed by {@link SegmentWriteOutMedium}, so it's own {@link #close()} method + * does nothing. However WriteOutBytes should appear closed, i. e. {@link #isOpen()} returns false, after the parental + * SegmentWriteOutMedium is closed. + */ +public abstract class WriteOutBytes extends OutputStream implements WritableByteChannel +{ + /** + * Writes 4 bytes of the given value in big-endian order, i. e. similar to {@link java.io.DataOutput#writeInt(int)}. + */ + public abstract void writeInt(int v) throws IOException; + + /** + * Returns the number of bytes written to this WriteOutBytes so far. + */ + public abstract long size() throws IOException; + + /** + * Takes all bytes that are written to this WriteOutBytes so far and writes them into the given channel. + */ + public abstract void writeTo(WritableByteChannel channel) throws IOException; + + /** + * Creates a finite {@link InputStream} with the bytes that are written to this WriteOutBytes so far. The returned + * InputStream must be closed properly after it's used up. + */ + public abstract InputStream asInputStream() throws IOException; + + /** + * Reads bytes from the byte sequences, represented by this WriteOutBytes, at the random position, into the given + * buffer. + * + * @throws java.nio.BufferUnderflowException if the byte sequence from the given pos ends before the given buffer + * is filled + * @throws IllegalArgumentException if the given pos is negative + */ + public abstract void readFully(long pos, ByteBuffer buffer) throws IOException; + + /** + * @deprecated does nothing. + */ + @Deprecated + @Override + public final void close() + { + // Does nothing. + } +} diff --git a/processing/src/test/java/io/druid/query/DoubleStorageTest.java b/processing/src/test/java/io/druid/query/DoubleStorageTest.java index 373fb88a2ec..d75eef3a759 100644 --- a/processing/src/test/java/io/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/io/druid/query/DoubleStorageTest.java @@ -32,6 +32,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.guava.Sequences; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.metadata.SegmentMetadataQueryConfig; import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest; @@ -110,8 +111,9 @@ public class DoubleStorageTest } - private static final IndexMergerV9 INDEX_MERGER_V9 = TestHelper.getTestIndexMergerV9(); - private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); + private static final IndexMergerV9 INDEX_MERGER_V9 = + TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); + private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()); private static final Integer MAX_ROWS = 10; private static final String TIME_COLUMN = "__time"; private static final String DIM_NAME = "testDimName"; @@ -346,7 +348,7 @@ public class DoubleStorageTest File someTmpFile = File.createTempFile("billy", "yay"); someTmpFile.delete(); someTmpFile.mkdirs(); - INDEX_MERGER_V9.persist(index, someTmpFile, new IndexSpec()); + INDEX_MERGER_V9.persist(index, someTmpFile, new IndexSpec(), null); someTmpFile.delete(); return INDEX_IO.loadIndex(someTmpFile); } diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index c7603797680..f3708608ba8 100644 --- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java @@ -35,6 +35,9 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregationTestHelper; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.dimension.DefaultDimensionSpec; @@ -61,8 +64,8 @@ import io.druid.segment.Segment; import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndex; import org.apache.commons.io.FileUtils; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -81,36 +84,38 @@ import java.util.Map; @RunWith(Parameterized.class) public class MultiValuedDimensionTest { - private AggregationTestHelper helper; + @Parameterized.Parameters(name = "{0}") + public static Collection constructorFeeder() throws IOException + { + final List constructors = Lists.newArrayList(); + for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { + constructors.add(new Object[]{config, TmpFileSegmentWriteOutMediumFactory.instance()}); + constructors.add(new Object[]{config, OffHeapMemorySegmentWriteOutMediumFactory.instance()}); + } + return constructors; + } - private static IncrementalIndex incrementalIndex; - private static QueryableIndex queryableIndex; + private final AggregationTestHelper helper; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - private static File persistedSegmentDir; + private IncrementalIndex incrementalIndex; + private QueryableIndex queryableIndex; - public MultiValuedDimensionTest( - final GroupByQueryConfig config - ) throws Exception + private File persistedSegmentDir; + + public MultiValuedDimensionTest(final GroupByQueryConfig config, SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + throws Exception { helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( ImmutableList.of(), config, null ); + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } - @Parameterized.Parameters(name = "{0}") - public static Collection constructorFeeder() throws IOException - { - final List constructors = Lists.newArrayList(); - for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - constructors.add(new Object[]{config}); - } - return constructors; - } - - @BeforeClass - public static void setupClass() throws Exception + @Before + public void setup() throws Exception { incrementalIndex = new IncrementalIndex.Builder() .setSimpleTestingIndexSchema(new CountAggregatorFactory("count")) @@ -141,10 +146,10 @@ public class MultiValuedDimensionTest } persistedSegmentDir = Files.createTempDir(); - TestHelper.getTestIndexMergerV9() - .persist(incrementalIndex, persistedSegmentDir, new IndexSpec()); + TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory) + .persist(incrementalIndex, persistedSegmentDir, new IndexSpec(), null); - queryableIndex = TestHelper.getTestIndexIO().loadIndex(persistedSegmentDir); + queryableIndex = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex(persistedSegmentDir); } @Test @@ -300,8 +305,8 @@ public class MultiValuedDimensionTest ); } - @AfterClass - public static void cleanup() throws Exception + @After + public void cleanup() throws Exception { queryableIndex.close(); incrementalIndex.close(); diff --git a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java index 7c76aeee97e..27e2c595856 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -46,6 +46,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.Yielder; import io.druid.java.util.common.guava.YieldingAccumulator; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -142,6 +143,7 @@ public class AggregationTestHelper IndexIO indexIO = new IndexIO( mapper, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -154,7 +156,7 @@ public class AggregationTestHelper return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, factory.getToolchest(), factory, @@ -197,6 +199,7 @@ public class AggregationTestHelper IndexIO indexIO = new IndexIO( mapper, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -209,7 +212,7 @@ public class AggregationTestHelper return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, toolchest, factory, @@ -237,6 +240,7 @@ public class AggregationTestHelper IndexIO indexIO = new IndexIO( mapper, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -249,7 +253,7 @@ public class AggregationTestHelper return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, toolchest, factory, @@ -288,6 +292,7 @@ public class AggregationTestHelper IndexIO indexIO = new IndexIO( mapper, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -300,7 +305,7 @@ public class AggregationTestHelper return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, toolchest, factory, @@ -429,7 +434,7 @@ public class AggregationTestHelper if (!index.canAppendRow()) { File tmp = tempFolder.newFolder(); toMerge.add(tmp); - indexMerger.persist(index, tmp, new IndexSpec()); + indexMerger.persist(index, tmp, new IndexSpec(), null); index.close(); index = new IncrementalIndex.Builder() .setIndexSchema( @@ -455,19 +460,19 @@ public class AggregationTestHelper if (toMerge.size() > 0) { File tmp = tempFolder.newFolder(); toMerge.add(tmp); - indexMerger.persist(index, tmp, new IndexSpec()); + indexMerger.persist(index, tmp, new IndexSpec(), null); List indexes = new ArrayList<>(toMerge.size()); for (File file : toMerge) { indexes.add(indexIO.loadIndex(file)); } - indexMerger.mergeQueryableIndex(indexes, true, metrics, outDir, new IndexSpec()); + indexMerger.mergeQueryableIndex(indexes, true, metrics, outDir, new IndexSpec(), null); for (QueryableIndex qi : indexes) { qi.close(); } } else { - indexMerger.persist(index, outDir, new IndexSpec()); + indexMerger.persist(index, outDir, new IndexSpec(), null); } } finally { diff --git a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java index 77f8dc6b7eb..41f49e5909d 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -47,8 +47,8 @@ import io.druid.segment.IdLookup; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.column.ValueType; -import io.druid.segment.data.ArrayBasedIndexedInts; import io.druid.segment.data.IndexedInts; +import io.druid.segment.data.SingleIndexedInt; import org.junit.Assert; import org.junit.Test; @@ -101,9 +101,9 @@ public class FilteredAggregatorTest public IndexedInts getRow() { if (selector.getIndex() % 3 == 2) { - return ArrayBasedIndexedInts.of(new int[]{1}); + return SingleIndexedInt.of(1); } else { - return ArrayBasedIndexedInts.of(new int[]{0}); + return SingleIndexedInt.of(0); } } diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index a1d95ca49d9..1daeaff48e4 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -48,6 +48,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.math.expr.ExprMacroTable; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.BySegmentQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; @@ -117,6 +118,7 @@ public class GroupByLimitPushDownInsufficientBufferTest ); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -126,7 +128,7 @@ public class GroupByLimitPushDownInsufficientBufferTest } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @@ -210,7 +212,8 @@ public class GroupByLimitPushDownInsufficientBufferTest final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - new IndexSpec() + new IndexSpec(), + OffHeapMemorySegmentWriteOutMediumFactory.instance() ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -251,7 +254,8 @@ public class GroupByLimitPushDownInsufficientBufferTest final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - new IndexSpec() + new IndexSpec(), + OffHeapMemorySegmentWriteOutMediumFactory.instance() ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java index cc117df0abc..d11f8e62d3f 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -49,6 +49,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.math.expr.ExprMacroTable; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.BySegmentQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; @@ -124,6 +125,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest ); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -133,7 +135,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private IncrementalIndex makeIncIndex(boolean withRollup) @@ -197,7 +199,8 @@ public class GroupByLimitPushDownMultiNodeMergeTest final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -232,7 +235,8 @@ public class GroupByLimitPushDownMultiNodeMergeTest final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); diff --git a/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java index 7f7e2bb3787..4df05bc9c68 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java @@ -46,6 +46,7 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.math.expr.ExprMacroTable; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.BySegmentQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; @@ -113,6 +114,7 @@ public class GroupByMultiSegmentTest ); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -122,7 +124,7 @@ public class GroupByMultiSegmentTest } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @@ -172,7 +174,8 @@ public class GroupByMultiSegmentTest final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -193,7 +196,8 @@ public class GroupByMultiSegmentTest final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index 4eb92b58e9e..39f659c99e3 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -27,6 +27,7 @@ import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; @@ -113,10 +114,11 @@ public class AppendTest @Before public void setUp() throws Exception { + SchemalessIndexTest schemalessIndexTest = new SchemalessIndexTest(OffHeapMemorySegmentWriteOutMediumFactory.instance()); // (1, 2) cover overlapping segments of the form // |------| // |--------| - QueryableIndex appendedIndex = SchemalessIndexTest.getAppendedIncrementalIndex( + QueryableIndex appendedIndex = schemalessIndexTest.getAppendedIncrementalIndex( Arrays.asList( new Pair("append.json.1", METRIC_AGGS_NO_UNIQ), new Pair("append.json.2", METRIC_AGGS) @@ -131,7 +133,7 @@ public class AppendTest // (3, 4) cover overlapping segments of the form // |------------| // |-----| - QueryableIndex append2 = SchemalessIndexTest.getAppendedIncrementalIndex( + QueryableIndex append2 = schemalessIndexTest.getAppendedIncrementalIndex( Arrays.asList( new Pair("append.json.3", METRIC_AGGS_NO_UNIQ), new Pair("append.json.4", METRIC_AGGS) @@ -147,7 +149,7 @@ public class AppendTest // |-------------| // |---| // |---| - QueryableIndex append3 = SchemalessIndexTest.getAppendedIncrementalIndex( + QueryableIndex append3 = schemalessIndexTest.getAppendedIncrementalIndex( Arrays.asList( new Pair("append.json.5", METRIC_AGGS), new Pair("append.json.6", METRIC_AGGS), diff --git a/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java index 55e6cb829a2..e3538fd70e2 100644 --- a/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java +++ b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java @@ -21,8 +21,9 @@ package io.druid.segment; import com.google.common.base.Function; import com.google.common.collect.Iterables; -import io.druid.segment.data.CompressedObjectStrategy; +import io.druid.java.util.common.io.Closer; import io.druid.segment.data.CompressedVSizeIndexedSupplierTest; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedMultivalue; import io.druid.segment.data.VSizeIndexedInts; @@ -30,12 +31,16 @@ import io.druid.segment.data.WritableSupplier; import org.junit.After; import org.junit.Before; +import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexedSupplierTest { + + private Closer closer; + @Override @Before public void setUpSimple() @@ -46,7 +51,7 @@ public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexed new int[]{6, 7, 8, 9, 10}, new int[]{11, 12, 13, 14, 15, 16, 17, 18, 19, 20} ); - + closer = Closer.create(); indexedSupplier = CompressedVSizeIndexedV3Supplier.fromIterable( Iterables.transform( vals, @@ -59,15 +64,17 @@ public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexed } } ), 2, 20, ByteOrder.nativeOrder(), - CompressedObjectStrategy.CompressionStrategy.LZ4 + CompressionStrategy.LZ4, + closer ); } @Override @After - public void teardown() + public void teardown() throws IOException { indexedSupplier = null; + closer.close(); vals = null; } @@ -76,8 +83,7 @@ public class CompressedVSizeIndexedV3SupplierTest extends CompressedVSizeIndexed { return CompressedVSizeIndexedV3Supplier.fromByteBuffer( buffer, - ByteOrder.nativeOrder(), - null + ByteOrder.nativeOrder() ); } } diff --git a/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java b/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java index 595a1e80e04..785f03c5272 100644 --- a/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java +++ b/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java @@ -19,8 +19,9 @@ package io.druid.segment; -import io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.data.CompressionFactory.LongEncodingStrategy; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -31,10 +32,17 @@ public class ConciseBitmapIndexMergerV9Test extends IndexMergerTestBase public ConciseBitmapIndexMergerV9Test( CompressionStrategy compressionStrategy, CompressionStrategy dimCompressionStrategy, - LongEncodingStrategy longEncodingStrategy + LongEncodingStrategy longEncodingStrategy, + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { - super(new ConciseBitmapSerdeFactory(), compressionStrategy, dimCompressionStrategy, longEncodingStrategy); - indexMerger = TestHelper.getTestIndexMergerV9(); + super( + new ConciseBitmapSerdeFactory(), + compressionStrategy, + dimCompressionStrategy, + longEncodingStrategy, + segmentWriteOutMediumFactory + ); + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); } } diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index 99654c6ff18..f7091e2bce4 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -19,10 +19,14 @@ package io.druid.segment; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.collections.bitmap.ConciseBitmapFactory; import io.druid.java.util.common.Intervals; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.Column; import io.druid.segment.incremental.IncrementalIndex; @@ -30,11 +34,33 @@ import io.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.commons.io.FileUtils; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; +import java.io.IOException; +import java.util.Collection; +@RunWith(Parameterized.class) public class EmptyIndexTest { + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} + ); + } + + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; + + public EmptyIndexTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + } + @Test public void testEmptyIndex() throws Exception { @@ -57,7 +83,7 @@ public class EmptyIndexTest emptyIndex, new ConciseBitmapFactory() ); - TestHelper.getTestIndexMergerV9().merge( + TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory).merge( Lists.newArrayList(emptyIndexAdapter), true, new AggregatorFactory[0], @@ -65,7 +91,7 @@ public class EmptyIndexTest new IndexSpec() ); - QueryableIndex emptyQueryableIndex = TestHelper.getTestIndexIO().loadIndex(tmpDir); + QueryableIndex emptyQueryableIndex = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex(tmpDir); Assert.assertEquals("getDimensionNames", 0, Iterables.size(emptyQueryableIndex.getAvailableDimensions())); Assert.assertEquals("getMetricNames", 0, Iterables.size(emptyQueryableIndex.getColumnNames())); diff --git a/processing/src/test/java/io/druid/segment/IndexBuilder.java b/processing/src/test/java/io/druid/segment/IndexBuilder.java index 990b4b21a40..a2213ac8d4d 100644 --- a/processing/src/test/java/io/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/io/druid/segment/IndexBuilder.java @@ -26,6 +26,8 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import io.druid.data.input.InputRow; import io.druid.java.util.common.StringUtils; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.incremental.IncrementalIndex; @@ -50,7 +52,8 @@ public class IndexBuilder private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() .withMetrics(new CountAggregatorFactory("count")) .build(); - private IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(); + private SegmentWriteOutMediumFactory segmentWriteOutMediumFactory = OffHeapMemorySegmentWriteOutMediumFactory.instance(); + private IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); private File tmpDir; private IndexSpec indexSpec = new IndexSpec(); private int maxRows = DEFAULT_MAX_ROWS; @@ -73,9 +76,10 @@ public class IndexBuilder return this; } - public IndexBuilder indexMerger(IndexMerger indexMerger) + public IndexBuilder segmentWriteOutMediumFactory(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - this.indexMerger = indexMerger; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); return this; } @@ -108,11 +112,12 @@ public class IndexBuilder Preconditions.checkNotNull(indexMerger, "indexMerger"); Preconditions.checkNotNull(tmpDir, "tmpDir"); try (final IncrementalIndex incrementalIndex = buildIncrementalIndex()) { - return TestHelper.getTestIndexIO().loadIndex( + return TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex( indexMerger.persist( incrementalIndex, new File(tmpDir, StringUtils.format("testIndex-%s", new Random().nextInt(Integer.MAX_VALUE))), - indexSpec + indexSpec, + null ) ); } @@ -123,14 +128,14 @@ public class IndexBuilder public QueryableIndex buildMMappedMergedIndex() { - Preconditions.checkNotNull(indexMerger, "indexMerger"); + IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); Preconditions.checkNotNull(tmpDir, "tmpDir"); final List persisted = Lists.newArrayList(); try { for (int i = 0; i < rows.size(); i += ROWS_PER_INDEX_FOR_MERGING) { persisted.add( - TestHelper.getTestIndexIO().loadIndex( + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex( indexMerger.persist( buildIncrementalIndexWithRows( schema, @@ -138,12 +143,13 @@ public class IndexBuilder rows.subList(i, Math.min(rows.size(), i + ROWS_PER_INDEX_FOR_MERGING)) ), new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID().toString())), - indexSpec + indexSpec, + null ) ) ); } - final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex( + final QueryableIndex merged = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex( indexMerger.merge( Lists.transform( persisted, diff --git a/processing/src/test/java/io/druid/segment/IndexIOTest.java b/processing/src/test/java/io/druid/segment/IndexIOTest.java index 429b603c53a..4fefaea2355 100644 --- a/processing/src/test/java/io/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/io/druid/segment/IndexIOTest.java @@ -31,10 +31,11 @@ import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.DimensionsSpec; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.UOE; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; @@ -66,8 +67,8 @@ public class IndexIOTest private static Interval DEFAULT_INTERVAL = Intervals.of("1970-01-01/2000-01-01"); private static final IndexSpec INDEX_SPEC = IndexMergerTestBase.makeIndexSpec( new ConciseBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionStrategy.LZ4, + CompressionStrategy.LZ4, CompressionFactory.LongEncodingStrategy.LONGS ); @@ -328,7 +329,7 @@ public class IndexIOTest { Exception ex = null; try { - TestHelper.getTestIndexIO().validateTwoSegments(adapter1, adapter2); + TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()).validateTwoSegments(adapter1, adapter2); } catch (Exception e) { ex = e; diff --git a/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java index dc094e569c8..278a1d75909 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerNullHandlingTest.java @@ -34,6 +34,7 @@ import io.druid.segment.column.Column; import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.incremental.IncrementalIndex; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -64,8 +65,8 @@ public class IndexMergerNullHandlingTest @Before public void setUp() { - indexMerger = TestHelper.getTestIndexMergerV9(); - indexIO = TestHelper.getTestIndexIO(); + indexMerger = TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); + indexIO = TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()); indexSpec = new IndexSpec(); } @@ -109,7 +110,7 @@ public class IndexMergerNullHandlingTest } final File tempDir = temporaryFolder.newFolder(); - try (QueryableIndex index = indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec))) { + try (QueryableIndex index = indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec, null))) { final Column column = index.getColumn("d"); if (subsetList.stream().allMatch(nullFlavors::contains)) { diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java index 6b9825a0a47..2434405651b 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java @@ -41,6 +41,7 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.ISE; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -50,8 +51,8 @@ import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.SimpleDictionaryEncodedColumn; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.BitmapValues; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexAdapter; @@ -81,22 +82,22 @@ import java.util.Map; public class IndexMergerTestBase { - private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); protected IndexMerger indexMerger; - @Parameterized.Parameters(name = "{index}: metric compression={0}, dimension compression={1}, long encoding={2}") + @Parameterized.Parameters(name = "{index}: metric compression={0}, dimension compression={1}, long encoding={2}, segment write-out medium={3}") public static Collection data() { return Collections2.transform( Sets.cartesianProduct( ImmutableList.of( - EnumSet.allOf(CompressedObjectStrategy.CompressionStrategy.class), - ImmutableSet.copyOf(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), - EnumSet.allOf(CompressionFactory.LongEncodingStrategy.class) + EnumSet.allOf(CompressionStrategy.class), + ImmutableSet.copyOf(CompressionStrategy.noNoneValues()), + EnumSet.allOf(CompressionFactory.LongEncodingStrategy.class), + SegmentWriteOutMediumFactory.builtInFactories() ) ), new Function, Object[]>() { @@ -112,8 +113,8 @@ public class IndexMergerTestBase static IndexSpec makeIndexSpec( BitmapSerdeFactory bitmapSerdeFactory, - CompressedObjectStrategy.CompressionStrategy compressionStrategy, - CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy, + CompressionStrategy compressionStrategy, + CompressionStrategy dimCompressionStrategy, CompressionFactory.LongEncodingStrategy longEncodingStrategy ) { @@ -130,17 +131,26 @@ public class IndexMergerTestBase } private final IndexSpec indexSpec; + private final IndexIO indexIO; + @Rule public final CloserRule closer = new CloserRule(false); protected IndexMergerTestBase( BitmapSerdeFactory bitmapSerdeFactory, - CompressedObjectStrategy.CompressionStrategy compressionStrategy, - CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy, - CompressionFactory.LongEncodingStrategy longEncodingStrategy + CompressionStrategy compressionStrategy, + CompressionStrategy dimCompressionStrategy, + CompressionFactory.LongEncodingStrategy longEncodingStrategy, + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { - this.indexSpec = makeIndexSpec(bitmapSerdeFactory, compressionStrategy, dimCompressionStrategy, longEncodingStrategy); + this.indexSpec = makeIndexSpec( + bitmapSerdeFactory, + compressionStrategy, + dimCompressionStrategy, + longEncodingStrategy + ); + this.indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); } @Test @@ -153,11 +163,12 @@ public class IndexMergerTestBase final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -200,11 +211,12 @@ public class IndexMergerTestBase final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -242,11 +254,12 @@ public class IndexMergerTestBase final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -302,11 +315,12 @@ public class IndexMergerTestBase final File mergedDir = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -316,11 +330,12 @@ public class IndexMergerTestBase Assert.assertEquals(3, index1.getColumnNames().size()); QueryableIndex index2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist2, tempDir2, - indexSpec + indexSpec, + null ) ) ); @@ -333,13 +348,14 @@ public class IndexMergerTestBase new CountAggregatorFactory("count") }; QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(index1, index2), true, mergedAggregators, mergedDir, - indexSpec + indexSpec, + null ) ) ); @@ -391,31 +407,34 @@ public class IndexMergerTestBase ); final QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tmpDir1, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex index2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist2, tmpDir2, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(index1, index2), true, new AggregatorFactory[]{}, tmpDir3, - indexSpec + indexSpec, + null ) ) ); @@ -451,11 +470,12 @@ public class IndexMergerTestBase ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -463,7 +483,7 @@ public class IndexMergerTestBase final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); - INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + indexIO.validateTwoSegments(incrementalAdapter, queryableAdapter); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); @@ -471,13 +491,14 @@ public class IndexMergerTestBase QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( ImmutableList.of(index1), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, mergedDir, - indexSpec + indexSpec, + null ) ) ); @@ -486,7 +507,7 @@ public class IndexMergerTestBase Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(3, merged.getColumnNames().size()); - INDEX_IO.validateTwoSegments(tempDir1, mergedDir); + indexIO.validateTwoSegments(tempDir1, mergedDir); assertDimCompression(index1, indexSpec.getDimensionCompression()); assertDimCompression(merged, indexSpec.getDimensionCompression()); @@ -509,15 +530,11 @@ public class IndexMergerTestBase ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( - indexMerger.append( - ImmutableList.of(incrementalAdapter), null, tempDir1, indexSpec - ) - ) + indexIO.loadIndex(indexMerger.append(ImmutableList.of(incrementalAdapter), null, tempDir1, indexSpec, null)) ); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); - INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + indexIO.validateTwoSegments(incrementalAdapter, queryableAdapter); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); @@ -530,13 +547,14 @@ public class IndexMergerTestBase AggregatorFactory[] mergedAggregators = new AggregatorFactory[]{new CountAggregatorFactory("count")}; QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( ImmutableList.of(index1), true, mergedAggregators, mergedDir, - indexSpec + indexSpec, + null ) ) ); @@ -545,7 +563,7 @@ public class IndexMergerTestBase Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(3, merged.getColumnNames().size()); - INDEX_IO.validateTwoSegments(tempDir1, mergedDir); + indexIO.validateTwoSegments(tempDir1, mergedDir); assertDimCompression(index1, indexSpec.getDimensionCompression()); assertDimCompression(merged, indexSpec.getDimensionCompression()); @@ -573,11 +591,12 @@ public class IndexMergerTestBase ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -585,7 +604,7 @@ public class IndexMergerTestBase final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); - INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + indexIO.validateTwoSegments(incrementalAdapter, queryableAdapter); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); @@ -593,12 +612,12 @@ public class IndexMergerTestBase IndexSpec newSpec = new IndexSpec( indexSpec.getBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? - CompressedObjectStrategy.CompressionStrategy.LZF : - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? - CompressedObjectStrategy.CompressionStrategy.LZF : - CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressionStrategy.LZF : + CompressionStrategy.LZ4, + CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressionStrategy.LZF : + CompressionStrategy.LZ4, CompressionFactory.LongEncodingStrategy.LONGS.equals(indexSpec.getLongEncoding()) ? CompressionFactory.LongEncodingStrategy.AUTO : CompressionFactory.LongEncodingStrategy.LONGS @@ -606,13 +625,14 @@ public class IndexMergerTestBase AggregatorFactory[] mergedAggregators = new AggregatorFactory[]{new CountAggregatorFactory("count")}; QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( ImmutableList.of(index1), true, mergedAggregators, mergedDir, - newSpec + newSpec, + null ) ) ); @@ -621,7 +641,7 @@ public class IndexMergerTestBase Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(merged.getAvailableDimensions())); Assert.assertEquals(3, merged.getColumnNames().size()); - INDEX_IO.validateTwoSegments(tempDir1, mergedDir); + indexIO.validateTwoSegments(tempDir1, mergedDir); assertDimCompression(index1, indexSpec.getDimensionCompression()); assertDimCompression(merged, newSpec.getDimensionCompression()); @@ -653,12 +673,12 @@ public class IndexMergerTestBase ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex(indexMerger.persist(toPersist1, tempDir1, indexSpec)) + indexIO.loadIndex(indexMerger.persist(toPersist1, tempDir1, indexSpec, null)) ); final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); - INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + indexIO.validateTwoSegments(incrementalAdapter, queryableAdapter); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); @@ -666,7 +686,7 @@ public class IndexMergerTestBase QueryableIndex converted = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.convert( tempDir1, convertDir, @@ -679,7 +699,7 @@ public class IndexMergerTestBase Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); Assert.assertEquals(4, converted.getColumnNames().size()); - INDEX_IO.validateTwoSegments(tempDir1, convertDir); + indexIO.validateTwoSegments(tempDir1, convertDir); assertDimCompression(index1, indexSpec.getDimensionCompression()); assertDimCompression(converted, indexSpec.getDimensionCompression()); @@ -716,11 +736,12 @@ public class IndexMergerTestBase ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -728,7 +749,7 @@ public class IndexMergerTestBase final IndexableAdapter queryableAdapter = new QueryableIndexIndexableAdapter(index1); - INDEX_IO.validateTwoSegments(incrementalAdapter, queryableAdapter); + indexIO.validateTwoSegments(incrementalAdapter, queryableAdapter); Assert.assertEquals(2, index1.getColumn(Column.TIME_COLUMN_NAME).getLength()); Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(index1.getAvailableDimensions())); @@ -737,19 +758,19 @@ public class IndexMergerTestBase IndexSpec newSpec = new IndexSpec( indexSpec.getBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? - CompressedObjectStrategy.CompressionStrategy.LZF : - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? - CompressedObjectStrategy.CompressionStrategy.LZF : - CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressionStrategy.LZF : + CompressionStrategy.LZ4, + CompressionStrategy.LZ4.equals(indexSpec.getDimensionCompression()) ? + CompressionStrategy.LZF : + CompressionStrategy.LZ4, CompressionFactory.LongEncodingStrategy.LONGS.equals(indexSpec.getLongEncoding()) ? CompressionFactory.LongEncodingStrategy.AUTO : CompressionFactory.LongEncodingStrategy.LONGS ); QueryableIndex converted = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.convert( tempDir1, convertDir, @@ -762,7 +783,7 @@ public class IndexMergerTestBase Assert.assertEquals(Arrays.asList("dim1", "dim2"), Lists.newArrayList(converted.getAvailableDimensions())); Assert.assertEquals(4, converted.getColumnNames().size()); - INDEX_IO.validateTwoSegments(tempDir1, convertDir); + indexIO.validateTwoSegments(tempDir1, convertDir); assertDimCompression(index1, indexSpec.getDimensionCompression()); assertDimCompression(converted, newSpec.getDimensionCompression()); @@ -773,11 +794,11 @@ public class IndexMergerTestBase ); } - private void assertDimCompression(QueryableIndex index, CompressedObjectStrategy.CompressionStrategy expectedStrategy) + private void assertDimCompression(QueryableIndex index, CompressionStrategy expectedStrategy) throws Exception { // Java voodoo - if (expectedStrategy == null || expectedStrategy == CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED) { + if (expectedStrategy == null || expectedStrategy == CompressionStrategy.UNCOMPRESSED) { return; } @@ -823,44 +844,48 @@ public class IndexMergerTestBase final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tmpDir, - indexSpec + indexSpec, + null ) ) ); QueryableIndex index2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist2, tmpDir2, - indexSpec + indexSpec, + null ) ) ); QueryableIndex index3 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist3, tmpDir3, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(index1, index2, index3), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); @@ -931,43 +956,47 @@ public class IndexMergerTestBase final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tmpDir, - indexSpec + indexSpec, + null ) ) ); QueryableIndex index2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist2, tmpDir2, - indexSpec + indexSpec, + null ) ) ); QueryableIndex index3 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist3, tmpDir3, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(index1, index2, index3), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); @@ -1012,55 +1041,60 @@ public class IndexMergerTestBase final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex indexA = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistA, tmpDirA, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexB = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistB, tmpDirB, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexB2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistB2, tmpDirB2, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(indexA, indexB), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(indexA, indexB2), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); @@ -1180,33 +1214,36 @@ public class IndexMergerTestBase final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex indexA = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistA, tmpDirA, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexB = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistB, tmpDirB, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(indexA, indexB), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); @@ -1322,33 +1359,36 @@ public class IndexMergerTestBase final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex indexA = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistA, tmpDirA, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexB = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistB, tmpDirB, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(indexA, indexB), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); @@ -1463,33 +1503,36 @@ public class IndexMergerTestBase final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex indexA = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistA, tmpDirA, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexB = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistB, tmpDirB, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(indexA, indexB), false, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); @@ -1574,75 +1617,82 @@ public class IndexMergerTestBase final File tmpDirMerged2 = temporaryFolder.newFolder(); QueryableIndex indexA = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistA, tmpDirA, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexB = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistB, tmpDirB, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexBA = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistBA, tmpDirBA, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexBA2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistBA2, tmpDirBA2, - indexSpec + indexSpec, + null ) ) ); QueryableIndex indexC = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersistC, tmpDirC, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(indexA, indexB, indexBA, indexBA2), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(indexA, indexB, indexBA, indexC), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged2, - indexSpec + indexSpec, + null ) ) ); @@ -1819,7 +1869,7 @@ public class IndexMergerTestBase tmpDirMerged, indexSpec ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(INDEX_IO.loadIndex( + final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( merged))); Assert.assertEquals(ImmutableSet.of("A", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); @@ -1890,7 +1940,7 @@ public class IndexMergerTestBase tmpDirMerged, indexSpec ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(INDEX_IO.loadIndex( + final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( merged))); Assert.assertEquals(ImmutableSet.of("A", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); @@ -1955,7 +2005,7 @@ public class IndexMergerTestBase ); // Since D was not present in any of the indices, it is not present in the output - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(INDEX_IO.loadIndex( + final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( merged))); Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); @@ -1997,7 +2047,7 @@ public class IndexMergerTestBase tmpDirMerged, indexSpec ); - final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(INDEX_IO.loadIndex( + final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(closer.closeLater(indexIO.loadIndex( merged))); Assert.assertEquals(ImmutableSet.of("A", "B", "C"), ImmutableSet.copyOf(adapter.getAvailableMetrics())); } @@ -2013,33 +2063,36 @@ public class IndexMergerTestBase final File tmpDirMerged = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tmpDir, - indexSpec + indexSpec, + null ) ) ); QueryableIndex index2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist2, tmpDir2, - indexSpec + indexSpec, + null ) ) ); final QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(index1, index2), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, tmpDirMerged, - indexSpec + indexSpec, + null ) ) ); @@ -2144,11 +2197,12 @@ public class IndexMergerTestBase final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( index1, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -2289,7 +2343,8 @@ public class IndexMergerTestBase indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ); } finally { @@ -2408,6 +2463,6 @@ public class IndexMergerTestBase } final File tempDir = temporaryFolder.newFolder(); - return closer.closeLater(INDEX_IO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec))); + return closer.closeLater(indexIO.loadIndex(indexMerger.persist(toPersist, tempDir, indexSpec, null))); } } diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java index 060bca51e44..aef1e55e268 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -27,10 +27,13 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.JodaUtils; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; 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.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.incremental.IncrementalIndex; import io.druid.segment.incremental.IncrementalIndexSchema; @@ -40,6 +43,8 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; @@ -52,10 +57,18 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +@RunWith(Parameterized.class) public class IndexMergerV9CompatibilityTest { - @Rule - public final CloserRule closer = new CloserRule(false); + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} + ); + } + private static final long TIMESTAMP = DateTimes.of("2014-01-01").getMillis(); private static final AggregatorFactory[] DEFAULT_AGG_FACTORIES = new AggregatorFactory[]{ new CountAggregatorFactory( @@ -63,21 +76,24 @@ public class IndexMergerV9CompatibilityTest ) }; - private static final IndexMergerV9 INDEX_MERGER_V9 = TestHelper.getTestIndexMergerV9(); - private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); - private static final IndexSpec INDEX_SPEC = IndexMergerTestBase.makeIndexSpec( new ConciseBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionStrategy.LZ4, + CompressionStrategy.LZ4, CompressionFactory.LongEncodingStrategy.LONGS ); private static final List DIMS = ImmutableList.of("dim0", "dim1"); private final Collection events; + @Rule + public final CloserRule closer = new CloserRule(false); + private final IndexMerger indexMerger; + private final IndexIO indexIO; - public IndexMergerV9CompatibilityTest() + public IndexMergerV9CompatibilityTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); events = new ArrayList<>(); final Map map1 = ImmutableMap.of( @@ -161,7 +177,7 @@ public class IndexMergerV9CompatibilityTest QueryableIndex index = null; try { outDir = Files.createTempDir(); - index = INDEX_IO.loadIndex(INDEX_MERGER_V9.persist(toPersist, outDir, INDEX_SPEC)); + index = indexIO.loadIndex(indexMerger.persist(toPersist, outDir, INDEX_SPEC, null)); Assert.assertEquals("value", index.getMetadata().get("key")); } @@ -181,7 +197,7 @@ public class IndexMergerV9CompatibilityTest { final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( persistTmpDir ) ) @@ -192,12 +208,12 @@ public class IndexMergerV9CompatibilityTest private File reprocessAndValidate(File inDir, File tmpDir) throws IOException { - final File outDir = INDEX_MERGER_V9.convert( + final File outDir = indexMerger.convert( inDir, tmpDir, INDEX_SPEC ); - INDEX_IO.validateTwoSegments(persistTmpDir, outDir); + indexIO.validateTwoSegments(persistTmpDir, outDir); return outDir; } @@ -206,7 +222,7 @@ public class IndexMergerV9CompatibilityTest { final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( persistTmpDir ) ) @@ -216,12 +232,12 @@ public class IndexMergerV9CompatibilityTest reprocessAndValidate(persistTmpDir, tmpDir1); final File tmpDir2 = new File(tmpDir, "reprocessed2"); - final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir1))); + final IndexableAdapter adapter2 = new QueryableIndexIndexableAdapter(closer.closeLater(indexIO.loadIndex(tmpDir1))); Assert.assertEquals(events.size(), adapter2.getNumRows()); reprocessAndValidate(tmpDir1, tmpDir2); final File tmpDir3 = new File(tmpDir, "reprocessed3"); - final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(INDEX_IO.loadIndex(tmpDir2))); + final IndexableAdapter adapter3 = new QueryableIndexIndexableAdapter(closer.closeLater(indexIO.loadIndex(tmpDir2))); Assert.assertEquals(events.size(), adapter3.getNumRows()); reprocessAndValidate(tmpDir2, tmpDir3); } diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java index 9c17dc9078d..7dca06cb57f 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -31,6 +31,7 @@ import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -56,6 +57,7 @@ import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -66,8 +68,6 @@ import java.util.Random; @RunWith(Parameterized.class) public class IndexMergerV9WithSpatialIndexTest { - private static IndexMergerV9 INDEX_MERGER_V9 = TestHelper.getTestIndexMergerV9(); - private static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); public static final int NUM_POINTS = 5000; private static Interval DATA_INTERVAL = Intervals.of("2013-01-01/2013-01-07"); @@ -82,23 +82,20 @@ public class IndexMergerV9WithSpatialIndexTest @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { - final IndexSpec indexSpec = new IndexSpec(); - final IncrementalIndex rtIndex = makeIncrementalIndex(); - final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec); - final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec); - return Arrays.asList( - new Object[][]{ - { - new IncrementalIndexSegment(rtIndex, null) - }, - { - new QueryableIndexSegment(null, mMappedTestIndex) - }, - { - new QueryableIndexSegment(null, mergedRealtimeIndex) - } - } - ); + List argumentArrays = new ArrayList<>(); + for (SegmentWriteOutMediumFactory segmentWriteOutMediumFactory : SegmentWriteOutMediumFactory.builtInFactories()) { + IndexMergerV9 indexMergerV9 = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + IndexIO indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); + + final IndexSpec indexSpec = new IndexSpec(); + final IncrementalIndex rtIndex = makeIncrementalIndex(); + final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec, indexMergerV9, indexIO); + final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec, indexMergerV9, indexIO); + argumentArrays.add(new Object[] {new IncrementalIndexSegment(rtIndex, null)}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mMappedTestIndex)}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedRealtimeIndex)}); + } + return argumentArrays; } private static IncrementalIndex makeIncrementalIndex() throws IOException @@ -255,7 +252,8 @@ public class IndexMergerV9WithSpatialIndexTest return theIndex; } - private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOException + private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec, IndexMergerV9 indexMergerV9, IndexIO indexIO) + throws IOException { IncrementalIndex theIndex = makeIncrementalIndex(); File tmpFile = File.createTempFile("billy", "yay"); @@ -263,15 +261,19 @@ public class IndexMergerV9WithSpatialIndexTest tmpFile.mkdirs(); try { - INDEX_MERGER_V9.persist(theIndex, tmpFile, indexSpec); - return INDEX_IO.loadIndex(tmpFile); + indexMergerV9.persist(theIndex, tmpFile, indexSpec, null); + return indexIO.loadIndex(tmpFile); } finally { FileUtils.deleteDirectory(tmpFile); } } - private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) + private static QueryableIndex makeMergedQueryableIndex( + IndexSpec indexSpec, + IndexMergerV9 indexMergerV9, + IndexIO indexIO + ) { try { IncrementalIndex first = new IncrementalIndex.Builder() @@ -493,22 +495,23 @@ public class IndexMergerV9WithSpatialIndexTest thirdFile.mkdirs(); mergedFile.mkdirs(); - INDEX_MERGER_V9.persist(first, DATA_INTERVAL, firstFile, indexSpec); - INDEX_MERGER_V9.persist(second, DATA_INTERVAL, secondFile, indexSpec); - INDEX_MERGER_V9.persist(third, DATA_INTERVAL, thirdFile, indexSpec); + indexMergerV9.persist(first, DATA_INTERVAL, firstFile, indexSpec, null); + indexMergerV9.persist(second, DATA_INTERVAL, secondFile, indexSpec, null); + indexMergerV9.persist(third, DATA_INTERVAL, thirdFile, indexSpec, null); try { - QueryableIndex mergedRealtime = INDEX_IO.loadIndex( - INDEX_MERGER_V9.mergeQueryableIndex( + QueryableIndex mergedRealtime = indexIO.loadIndex( + indexMergerV9.mergeQueryableIndex( Arrays.asList( - INDEX_IO.loadIndex(firstFile), - INDEX_IO.loadIndex(secondFile), - INDEX_IO.loadIndex(thirdFile) + indexIO.loadIndex(firstFile), + indexIO.loadIndex(secondFile), + indexIO.loadIndex(thirdFile) ), true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); return mergedRealtime; diff --git a/processing/src/test/java/io/druid/segment/IndexSpecTest.java b/processing/src/test/java/io/druid/segment/IndexSpecTest.java index 68223b5680c..3fbd76e0c15 100644 --- a/processing/src/test/java/io/druid/segment/IndexSpecTest.java +++ b/processing/src/test/java/io/druid/segment/IndexSpecTest.java @@ -21,8 +21,8 @@ package io.druid.segment; 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.CompressionStrategy; import io.druid.segment.data.RoaringBitmapSerdeFactory; import org.junit.Assert; import org.junit.Test; @@ -38,8 +38,8 @@ public class IndexSpecTest final IndexSpec spec = objectMapper.readValue(json, IndexSpec.class); Assert.assertEquals(new RoaringBitmapSerdeFactory(null), spec.getBitmapSerdeFactory()); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompression()); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZF, spec.getMetricCompression()); + Assert.assertEquals(CompressionStrategy.LZ4, spec.getDimensionCompression()); + Assert.assertEquals(CompressionStrategy.LZF, spec.getMetricCompression()); Assert.assertEquals(CompressionFactory.LongEncodingStrategy.AUTO, spec.getLongEncoding()); Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class)); @@ -53,7 +53,7 @@ public class IndexSpecTest final IndexSpec spec = objectMapper.readValue(json, IndexSpec.class); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, spec.getDimensionCompression()); + Assert.assertEquals(CompressionStrategy.UNCOMPRESSED, spec.getDimensionCompression()); Assert.assertEquals(spec, objectMapper.readValue(objectMapper.writeValueAsBytes(spec), IndexSpec.class)); } @@ -61,8 +61,8 @@ public class IndexSpecTest public void testDefaults() throws Exception { final IndexSpec spec = new IndexSpec(); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getDimensionCompression()); - Assert.assertEquals(CompressedObjectStrategy.CompressionStrategy.LZ4, spec.getMetricCompression()); + Assert.assertEquals(CompressionStrategy.LZ4, spec.getDimensionCompression()); + Assert.assertEquals(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 2f6a1202e03..3fbf954cba8 100644 --- a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java @@ -19,9 +19,13 @@ package io.druid.segment; +import com.google.common.collect.ImmutableList; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.segment.data.BitmapValues; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.incremental.IncrementalIndex; @@ -29,25 +33,47 @@ import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; +import java.io.IOException; +import java.util.Collection; +@RunWith(Parameterized.class) public class QueryableIndexIndexableAdapterTest { - private final static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); - private final static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); private static final IndexSpec INDEX_SPEC = IndexMergerTestBase.makeIndexSpec( new ConciseBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionStrategy.LZ4, + CompressionStrategy.LZ4, CompressionFactory.LongEncodingStrategy.LONGS ); + + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} + ); + } + @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder(); @Rule public final CloserRule closer = new CloserRule(false); + private final IndexMerger indexMerger; + private final IndexIO indexIO; + + public QueryableIndexIndexableAdapterTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); + } + @Test public void testGetBitmapIndex() throws Exception { @@ -57,18 +83,19 @@ public class QueryableIndexIndexableAdapterTest final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( - INDEX_MERGER.persist( + indexIO.loadIndex( + indexMerger.persist( toPersist, tempDir, - INDEX_SPEC + INDEX_SPEC, + null ) ) ); IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); String dimension = "dim1"; - //null is added to all dimensions with value + @SuppressWarnings("UnusedAssignment") //null is added to all dimensions with value BitmapValues bitmapValues = adapter.getBitmapValues(dimension, 0); for (int i = 0; i < adapter.getDimValueLookup(dimension).size(); i++) { bitmapValues = adapter.getBitmapValues(dimension, i); diff --git a/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java b/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java index c5e59470be9..403c5f54ebd 100644 --- a/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java +++ b/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java @@ -19,8 +19,9 @@ package io.druid.segment; -import io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.data.CompressionFactory.LongEncodingStrategy; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.RoaringBitmapSerdeFactory; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -31,10 +32,17 @@ public class RoaringBitmapIndexMergerV9Test extends IndexMergerTestBase public RoaringBitmapIndexMergerV9Test( CompressionStrategy compressionStrategy, CompressionStrategy dimCompressionStrategy, - LongEncodingStrategy longEncodingStrategy + LongEncodingStrategy longEncodingStrategy, + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { - super(new RoaringBitmapSerdeFactory(null), compressionStrategy, dimCompressionStrategy, longEncodingStrategy); - indexMerger = TestHelper.getTestIndexMergerV9(); + super( + new RoaringBitmapSerdeFactory(null), + compressionStrategy, + dimCompressionStrategy, + longEncodingStrategy, + segmentWriteOutMediumFactory + ); + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); } } diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java index ca2caca63f2..f31c4a75f3e 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java @@ -35,6 +35,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -89,9 +90,6 @@ public class SchemalessIndexTest private static final Map> mergedIndexes = Maps.newHashMap(); private static final List rowPersistedIndexes = Lists.newArrayList(); - private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); - private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); - private static IncrementalIndex index = null; private static QueryableIndex mergedIndex = null; @@ -101,6 +99,15 @@ public class SchemalessIndexTest } } + private final IndexMerger indexMerger; + private final IndexIO indexIO; + + public SchemalessIndexTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); + } + public static IncrementalIndex getIncrementalIndex() { synchronized (log) { @@ -178,7 +185,7 @@ public class SchemalessIndexTest } } - public static QueryableIndex getMergedIncrementalIndex() + public QueryableIndex getMergedIncrementalIndex() { synchronized (log) { if (mergedIndex != null) { @@ -203,16 +210,17 @@ public class SchemalessIndexTest mergedFile.mkdirs(); mergedFile.deleteOnExit(); - INDEX_MERGER.persist(top, topFile, indexSpec); - INDEX_MERGER.persist(bottom, bottomFile, indexSpec); + indexMerger.persist(top, topFile, indexSpec, null); + indexMerger.persist(bottom, bottomFile, indexSpec, null); - mergedIndex = INDEX_IO.loadIndex( - INDEX_MERGER.mergeQueryableIndex( - Arrays.asList(INDEX_IO.loadIndex(topFile), INDEX_IO.loadIndex(bottomFile)), + mergedIndex = indexIO.loadIndex( + indexMerger.mergeQueryableIndex( + Arrays.asList(indexIO.loadIndex(topFile), indexIO.loadIndex(bottomFile)), true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); @@ -225,7 +233,7 @@ public class SchemalessIndexTest } } - public static QueryableIndex getMergedIncrementalIndex(int index1, int index2) + public QueryableIndex getMergedIncrementalIndex(int index1, int index2) { synchronized (log) { if (rowPersistedIndexes.isEmpty()) { @@ -252,13 +260,14 @@ public class SchemalessIndexTest mergedFile.mkdirs(); mergedFile.deleteOnExit(); - QueryableIndex index = INDEX_IO.loadIndex( - INDEX_MERGER.mergeQueryableIndex( + QueryableIndex index = indexIO.loadIndex( + indexMerger.mergeQueryableIndex( Arrays.asList(rowPersistedIndexes.get(index1), rowPersistedIndexes.get(index2)), true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); @@ -272,7 +281,7 @@ public class SchemalessIndexTest } } - public static QueryableIndex getMergedIncrementalIndex(int[] indexes) + public QueryableIndex getMergedIncrementalIndex(int[] indexes) { synchronized (log) { if (rowPersistedIndexes.isEmpty()) { @@ -293,8 +302,8 @@ public class SchemalessIndexTest indexesToMerge.add(rowPersistedIndexes.get(index)); } - return INDEX_IO.loadIndex( - INDEX_MERGER.mergeQueryableIndex(indexesToMerge, true, METRIC_AGGS, mergedFile, indexSpec) + return indexIO.loadIndex( + indexMerger.mergeQueryableIndex(indexesToMerge, true, METRIC_AGGS, mergedFile, indexSpec, null) ); } catch (IOException e) { @@ -303,7 +312,7 @@ public class SchemalessIndexTest } } - public static QueryableIndex getAppendedIncrementalIndex( + public QueryableIndex getAppendedIncrementalIndex( Iterable> files, List intervals ) @@ -311,7 +320,7 @@ public class SchemalessIndexTest return makeAppendedMMappedIndex(files, intervals); } - public static QueryableIndex getMergedIncrementalIndexDiffMetrics() + public QueryableIndex getMergedIncrementalIndexDiffMetrics() { return getMergedIncrementalIndex( Arrays.>asList( @@ -321,7 +330,7 @@ public class SchemalessIndexTest ); } - public static QueryableIndex getMergedIncrementalIndex(Iterable> files) + public QueryableIndex getMergedIncrementalIndex(Iterable> files) { return makeMergedMMappedIndex(files); } @@ -342,7 +351,7 @@ public class SchemalessIndexTest } } - private static void makeRowPersistedIndexes() + private void makeRowPersistedIndexes() { synchronized (log) { try { @@ -380,8 +389,8 @@ public class SchemalessIndexTest tmpFile.mkdirs(); tmpFile.deleteOnExit(); - INDEX_MERGER.persist(rowIndex, tmpFile, indexSpec); - rowPersistedIndexes.add(INDEX_IO.loadIndex(tmpFile)); + indexMerger.persist(rowIndex, tmpFile, indexSpec, null); + rowPersistedIndexes.add(indexIO.loadIndex(tmpFile)); } } catch (IOException e) { @@ -437,8 +446,7 @@ public class SchemalessIndexTest return retVal; } - private static List makeFilesToMap(File tmpFile, Iterable> files) - throws IOException + private List makeFilesToMap(File tmpFile, Iterable> files) throws IOException { List filesToMap = Lists.newArrayList(); for (Pair file : files) { @@ -447,13 +455,13 @@ public class SchemalessIndexTest theFile.mkdirs(); theFile.deleteOnExit(); filesToMap.add(theFile); - INDEX_MERGER.persist(index, theFile, indexSpec); + indexMerger.persist(index, theFile, indexSpec, null); } return filesToMap; } - private static QueryableIndex makeAppendedMMappedIndex( + private QueryableIndex makeAppendedMMappedIndex( Iterable> files, final List intervals ) @@ -498,7 +506,7 @@ public class SchemalessIndexTest { try { return new RowboatFilteringIndexAdapter( - new QueryableIndexIndexableAdapter(INDEX_IO.loadIndex(chunk.getObject())), + new QueryableIndexIndexableAdapter(indexIO.loadIndex(chunk.getObject())), new Predicate() { @Override @@ -521,14 +529,14 @@ public class SchemalessIndexTest ) ); - return INDEX_IO.loadIndex(INDEX_MERGER.append(adapters, null, mergedFile, indexSpec)); + return indexIO.loadIndex(indexMerger.append(adapters, null, mergedFile, indexSpec, null)); } catch (IOException e) { throw Throwables.propagate(e); } } - private static QueryableIndex makeMergedMMappedIndex(Iterable> files) + private QueryableIndex makeMergedMMappedIndex(Iterable> files) { try { File tmpFile = File.createTempFile("yay", "who"); @@ -539,8 +547,8 @@ public class SchemalessIndexTest List filesToMap = makeFilesToMap(tmpFile, files); - return INDEX_IO.loadIndex( - INDEX_MERGER.mergeQueryableIndex( + return indexIO.loadIndex( + indexMerger.mergeQueryableIndex( Lists.newArrayList( Iterables.transform( filesToMap, @@ -550,7 +558,7 @@ public class SchemalessIndexTest public QueryableIndex apply(@Nullable File input) { try { - return INDEX_IO.loadIndex(input); + return indexIO.loadIndex(input); } catch (IOException e) { throw Throwables.propagate(e); @@ -562,7 +570,8 @@ public class SchemalessIndexTest true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); } diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java index 088197341fd..7e6f6cb627a 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java @@ -19,6 +19,7 @@ package io.druid.segment; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; @@ -29,6 +30,9 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Sequences; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; @@ -58,8 +62,12 @@ import io.druid.query.topn.TopNQuery; import io.druid.query.topn.TopNQueryBuilder; import io.druid.query.topn.TopNResultValue; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import java.io.IOException; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -67,11 +75,22 @@ import java.util.Map; /** */ +@RunWith(Parameterized.class) public class SchemalessTestFullTest { + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} + ); + } + final double UNIQUES_2 = 2.000977198748901d; final double UNIQUES_1 = 1.0002442201269182d; + final SchemalessIndexTest schemalessIndexTest; final String dataSource = "testing"; final Granularity allGran = Granularities.ALL; final String marketDimension = "market"; @@ -95,6 +114,11 @@ public class SchemalessTestFullTest Arrays.asList(Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) ); + public SchemalessTestFullTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + schemalessIndexTest = new SchemalessIndexTest(segmentWriteOutMediumFactory); + } + @Test public void testCompleteIntersectingSchemas() { @@ -927,7 +951,7 @@ public class SchemalessTestFullTest runTests( new QueryableIndexSegment( - null, SchemalessIndexTest.getMergedIncrementalIndex(0, 0) + null, schemalessIndexTest.getMergedIncrementalIndex(0, 0) ), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, @@ -1012,7 +1036,7 @@ public class SchemalessTestFullTest runTests( new QueryableIndexSegment( - null, SchemalessIndexTest.getMergedIncrementalIndex(1, 1) + null, schemalessIndexTest.getMergedIncrementalIndex(1, 1) ), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, @@ -1143,7 +1167,7 @@ public class SchemalessTestFullTest ); runTests( - new QueryableIndexSegment(null, SchemalessIndexTest.getMergedIncrementalIndex(new int[]{6, 7, 8})), + new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndex(new int[]{6, 7, 8})), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, expectedTopNResults, @@ -1334,7 +1358,7 @@ public class SchemalessTestFullTest ); runTests( - new QueryableIndexSegment(null, SchemalessIndexTest.getMergedIncrementalIndexDiffMetrics()), + new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndexDiffMetrics()), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, expectedTopNResults, @@ -1359,11 +1383,11 @@ public class SchemalessTestFullTest StringUtils.format("Failed: II[%,d, %,d]", index2, index1) ), new Pair<>( - SchemalessIndexTest.getMergedIncrementalIndex(index1, index2), + schemalessIndexTest.getMergedIncrementalIndex(index1, index2), StringUtils.format("Failed: MII[%,d, %,d]", index1, index2) ), new Pair<>( - SchemalessIndexTest.getMergedIncrementalIndex(index2, index1), + schemalessIndexTest.getMergedIncrementalIndex(index2, index1), StringUtils.format("Failed: MII[%,d, %,d]", index2, index1) ) ); diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java index 353b03081ff..2c04eca92b7 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java @@ -26,6 +26,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; @@ -60,6 +61,7 @@ import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; @@ -73,27 +75,17 @@ public class SchemalessTestSimpleTest @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { - final IncrementalIndex incrementalIndex = SchemalessIndexTest.getIncrementalIndex(); - final QueryableIndex persistedIncrementalIndex = TestIndex.persistRealtimeAndLoadMMapped(incrementalIndex); - final QueryableIndex mergedIncrementalIndex = SchemalessIndexTest.getMergedIncrementalIndex(); - - return Arrays.asList( - new Object[][]{ - { - new IncrementalIndexSegment(incrementalIndex, null) - }, - { - new QueryableIndexSegment( - null, persistedIncrementalIndex - ) - }, - { - new QueryableIndexSegment( - null, mergedIncrementalIndex - ) - } - } - ); + List argumentArrays = new ArrayList<>(); + for (SegmentWriteOutMediumFactory segmentWriteOutMediumFactory : SegmentWriteOutMediumFactory.builtInFactories()) { + SchemalessIndexTest schemalessIndexTest = new SchemalessIndexTest(segmentWriteOutMediumFactory); + final IncrementalIndex incrementalIndex = SchemalessIndexTest.getIncrementalIndex(); + final QueryableIndex persistedIncrementalIndex = TestIndex.persistRealtimeAndLoadMMapped(incrementalIndex); + final QueryableIndex mergedIncrementalIndex = schemalessIndexTest.getMergedIncrementalIndex(); + argumentArrays.add(new Object[] {new IncrementalIndexSegment(incrementalIndex, null)}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, persistedIncrementalIndex)}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedIncrementalIndex)}); + } + return argumentArrays; } final String dataSource = "testing"; diff --git a/processing/src/test/java/io/druid/segment/StringDimensionHandlerTest.java b/processing/src/test/java/io/druid/segment/StringDimensionHandlerTest.java index 6103abfd867..acc9abd5793 100644 --- a/processing/src/test/java/io/druid/segment/StringDimensionHandlerTest.java +++ b/processing/src/test/java/io/druid/segment/StringDimensionHandlerTest.java @@ -25,8 +25,8 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.query.aggregation.CountAggregatorFactory; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.Indexed; import io.druid.segment.incremental.IncrementalIndex; @@ -49,8 +49,8 @@ public class StringDimensionHandlerTest private static final IndexSpec INDEX_SPEC = new IndexSpec( new ConciseBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionStrategy.LZ4, + CompressionStrategy.LZ4, CompressionFactory.LongEncodingStrategy.LONGS ); diff --git a/processing/src/test/java/io/druid/segment/TestHelper.java b/processing/src/test/java/io/druid/segment/TestHelper.java index 5ca38930514..7da66da3b59 100644 --- a/processing/src/test/java/io/druid/segment/TestHelper.java +++ b/processing/src/test/java/io/druid/segment/TestHelper.java @@ -29,6 +29,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.math.expr.ExprMacroTable; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Result; import io.druid.query.expression.TestExprMacroTable; import io.druid.query.timeseries.TimeseriesResultValue; @@ -46,13 +47,18 @@ import java.util.stream.IntStream; */ public class TestHelper { - private static final IndexMergerV9 INDEX_MERGER_V9; - private static final IndexIO INDEX_IO; + private static final ObjectMapper JSON_MAPPER = getJsonMapper(); - static { - final ObjectMapper jsonMapper = getJsonMapper(); - INDEX_IO = new IndexIO( - jsonMapper, + public static IndexMergerV9 getTestIndexMergerV9(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + return new IndexMergerV9(JSON_MAPPER, getTestIndexIO(segmentWriteOutMediumFactory), segmentWriteOutMediumFactory); + } + + public static IndexIO getTestIndexIO(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + return new IndexIO( + JSON_MAPPER, + segmentWriteOutMediumFactory, new ColumnConfig() { @Override @@ -62,17 +68,6 @@ public class TestHelper } } ); - INDEX_MERGER_V9 = new IndexMergerV9(jsonMapper, INDEX_IO); - } - - public static IndexMergerV9 getTestIndexMergerV9() - { - return INDEX_MERGER_V9; - } - - public static IndexIO getTestIndexIO() - { - return INDEX_IO; } public static ObjectMapper getJsonMapper() diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index 4271eaa213b..a6b03a6dd0d 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -37,6 +37,7 @@ import io.druid.hll.HyperLogLogHash; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.logger.Logger; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.DoubleMinAggregatorFactory; @@ -135,8 +136,9 @@ public class TestIndex }; private static final IndexSpec indexSpec = new IndexSpec(); - private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); - private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); + private static final IndexMerger INDEX_MERGER = + TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); + private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()); static { if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { @@ -225,8 +227,8 @@ public class TestIndex mergedFile.mkdirs(); mergedFile.deleteOnExit(); - INDEX_MERGER.persist(top, DATA_INTERVAL, topFile, indexSpec); - INDEX_MERGER.persist(bottom, DATA_INTERVAL, bottomFile, indexSpec); + INDEX_MERGER.persist(top, DATA_INTERVAL, topFile, indexSpec, null); + INDEX_MERGER.persist(bottom, DATA_INTERVAL, bottomFile, indexSpec, null); mergedRealtime = INDEX_IO.loadIndex( INDEX_MERGER.mergeQueryableIndex( @@ -234,7 +236,8 @@ public class TestIndex true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); @@ -362,7 +365,7 @@ public class TestIndex someTmpFile.mkdirs(); someTmpFile.deleteOnExit(); - INDEX_MERGER.persist(index, someTmpFile, indexSpec); + INDEX_MERGER.persist(index, someTmpFile, indexSpec, null); return INDEX_IO.loadIndex(someTmpFile); } catch (IOException e) { diff --git a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java index 16c58c01a24..5eaa9e55a29 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java @@ -20,10 +20,10 @@ package io.druid.segment.data; import com.google.common.base.Supplier; -import com.google.common.io.ByteSink; import com.google.common.primitives.Floats; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.Assert; import org.junit.Test; @@ -32,7 +32,6 @@ 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; @@ -50,7 +49,7 @@ public class CompressedFloatsSerdeTest public static Iterable compressionStrategies() { List data = new ArrayList<>(); - for (CompressedObjectStrategy.CompressionStrategy strategy : CompressedObjectStrategy.CompressionStrategy.values()) { + for (CompressionStrategy strategy : CompressionStrategy.values()) { data.add(new Object[]{strategy, ByteOrder.BIG_ENDIAN}); data.add(new Object[]{strategy, ByteOrder.LITTLE_ENDIAN}); } @@ -59,7 +58,7 @@ public class CompressedFloatsSerdeTest private static final double DELTA = 0.00001; - protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + protected final CompressionStrategy compressionStrategy; protected final ByteOrder order; private final float values0[] = {}; @@ -75,7 +74,7 @@ public class CompressedFloatsSerdeTest }; public CompressedFloatsSerdeTest( - CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressionStrategy compressionStrategy, ByteOrder order ) { @@ -108,7 +107,11 @@ public class CompressedFloatsSerdeTest public void testWithValues(float[] values) throws Exception { - FloatSupplierSerializer serializer = CompressionFactory.getFloatSerializer(new IOPeonForTesting(), "test", order, compressionStrategy + FloatSupplierSerializer serializer = CompressionFactory.getFloatSerializer( + new OffHeapMemorySegmentWriteOutMedium(), + "test", + order, + compressionStrategy ); serializer.open(); @@ -118,19 +121,10 @@ public class CompressedFloatsSerdeTest Assert.assertEquals(values.length, serializer.size()); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.closeAndConsolidate( - new ByteSink() - { - @Override - public OutputStream openStream() throws IOException - { - return baos; - } - } - ); + serializer.writeTo(Channels.newChannel(baos), null); Assert.assertEquals(baos.size(), serializer.getSerializedSize()); CompressedFloatsIndexedSupplier supplier = CompressedFloatsIndexedSupplier - .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order, null); + .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order); IndexedFloats floats = supplier.get(); assertIndexMatchesVals(floats, values); @@ -180,12 +174,12 @@ public class CompressedFloatsSerdeTest private void testSupplierSerde(CompressedFloatsIndexedSupplier supplier, float[] vals) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - supplier.writeToChannel(Channels.newChannel(baos)); + supplier.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); Assert.assertEquals(supplier.getSerializedSize(), bytes.length); CompressedFloatsIndexedSupplier anotherSupplier = CompressedFloatsIndexedSupplier.fromByteBuffer( - ByteBuffer.wrap(bytes), order, null + ByteBuffer.wrap(bytes), order ); IndexedFloats indexed = anotherSupplier.get(); assertIndexMatchesVals(indexed, vals); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java index 455b1af0d31..adc3f8c40c3 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedSupplierTest.java @@ -23,6 +23,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.java.util.common.io.Closer; import io.druid.segment.CompressedPools; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.After; @@ -44,11 +45,12 @@ import java.util.concurrent.atomic.AtomicReference; public class CompressedIntsIndexedSupplierTest extends CompressionStrategyTest { - public CompressedIntsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + public CompressedIntsIndexedSupplierTest(CompressionStrategy compressionStrategy) { super(compressionStrategy); } + private Closer closer; private IndexedInts indexed; private CompressedIntsIndexedSupplier supplier; private int[] vals; @@ -56,6 +58,7 @@ public class CompressedIntsIndexedSupplierTest extends CompressionStrategyTest @Before public void setUp() throws Exception { + closer = Closer.create(); CloseQuietly.close(indexed); indexed = null; supplier = null; @@ -65,6 +68,7 @@ public class CompressedIntsIndexedSupplierTest extends CompressionStrategyTest @After public void tearDown() throws Exception { + closer.close(); CloseQuietly.close(indexed); } @@ -78,7 +82,8 @@ public class CompressedIntsIndexedSupplierTest extends CompressionStrategyTest IntBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), - compressionStrategy + compressionStrategy, + closer ); indexed = supplier.get(); @@ -97,14 +102,14 @@ public class CompressedIntsIndexedSupplierTest extends CompressionStrategyTest ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedIntsIndexedSupplier theSupplier = CompressedIntsIndexedSupplier.fromIntBuffer( - IntBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy + IntBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), compressionStrategy, closer ); - theSupplier.writeToChannel(Channels.newChannel(baos)); + theSupplier.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length); - supplier = CompressedIntsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), ByteOrder.nativeOrder(), null); + supplier = CompressedIntsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), ByteOrder.nativeOrder()); indexed = supplier.get(); } 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 303f10c8b9c..3a2441e4f51 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java @@ -22,7 +22,6 @@ package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; @@ -30,6 +29,10 @@ import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import it.unimi.dsi.fastutil.ints.IntArrayList; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.junit.After; @@ -42,8 +45,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; import java.nio.file.Files; import java.util.List; import java.util.Random; @@ -56,13 +57,14 @@ public class CompressedIntsIndexedWriterTest { private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; private static final int[] CHUNK_FACTORS = new int[]{1, 2, 100, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER}; - private final IOPeon ioPeon = new TmpFileIOPeon(); - private final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + private final SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); + private final CompressionStrategy compressionStrategy; private final ByteOrder byteOrder; private final Random rand = new Random(0); private int[] vals; + public CompressedIntsIndexedWriterTest( - CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressionStrategy compressionStrategy, ByteOrder byteOrder ) { @@ -74,7 +76,7 @@ public class CompressedIntsIndexedWriterTest public static Iterable compressionStrategiesAndByteOrders() { Set> combinations = Sets.cartesianProduct( - Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), + Sets.newHashSet(CompressionStrategy.noNoneValues()), Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) ); @@ -99,7 +101,7 @@ public class CompressedIntsIndexedWriterTest @After public void tearDown() throws Exception { - ioPeon.close(); + segmentWriteOutMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -115,29 +117,30 @@ public class CompressedIntsIndexedWriterTest FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( - ioPeon, "test", chunkFactor, byteOrder, compressionStrategy + segmentWriteOutMedium, "test", chunkFactor, byteOrder, compressionStrategy ); CompressedIntsIndexedSupplier supplierFromList = CompressedIntsIndexedSupplier.fromList( - Ints.asList(vals), chunkFactor, byteOrder, compressionStrategy + IntArrayList.wrap(vals), + chunkFactor, + byteOrder, + compressionStrategy, + segmentWriteOutMedium.getCloser() ); writer.open(); for (int val : vals) { writer.add(val); } - writer.close(); long writtenLength = writer.getSerializedSize(); - final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); - writer.writeToChannel(outputChannel, smoosher); - outputChannel.close(); + final WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + writer.writeTo(writeOutBytes, smoosher); smoosher.close(); assertEquals(writtenLength, supplierFromList.getSerializedSize()); // read from ByteBuffer and check values CompressedIntsIndexedSupplier supplierFromByteBuffer = CompressedIntsIndexedSupplier.fromByteBuffer( - ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), - byteOrder, - null + ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())), + byteOrder ); IndexedInts indexedInts = supplierFromByteBuffer.get(); assertEquals(vals.length, indexedInts.size()); @@ -187,51 +190,43 @@ public class CompressedIntsIndexedWriterTest )).toFile(); FileSmoosher smoosher = new FileSmoosher(tmpDirectory); - final IOPeon ioPeon = new TmpFileIOPeon(); - try { - CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( - chunkFactor, - compressionStrategy, - new GenericIndexedWriter<>( - ioPeon, "test", - CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compressionStrategy, - chunkFactor - ), Longs.BYTES * 10000 - ) - ); + CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( + segmentWriteOutMedium, + chunkFactor, + byteOrder, + compressionStrategy, + GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, + "test", + compressionStrategy, + Longs.BYTES * 10000 + ) + ); - writer.open(); - for (int val : vals) { - writer.add(val); - } - writer.close(); - final SmooshedWriter channel = smoosher.addWithSmooshedWriter( - "test", writer.getSerializedSize() - ); - writer.writeToChannel(channel, smoosher); - channel.close(); - smoosher.close(); - - SmooshedFileMapper mapper = Smoosh.map(tmpDirectory); - - // read from ByteBuffer and check values - CompressedIntsIndexedSupplier supplierFromByteBuffer = CompressedIntsIndexedSupplier.fromByteBuffer( - mapper.mapFile("test"), - byteOrder, - mapper - ); - IndexedInts indexedInts = supplierFromByteBuffer.get(); - assertEquals(vals.length, indexedInts.size()); - for (int i = 0; i < vals.length; ++i) { - assertEquals(vals[i], indexedInts.get(i)); - } - CloseQuietly.close(indexedInts); - mapper.close(); + writer.open(); + for (int val : vals) { + writer.add(val); } - finally { - ioPeon.close(); + final SmooshedWriter channel = smoosher.addWithSmooshedWriter("test", writer.getSerializedSize()); + writer.writeTo(channel, smoosher); + channel.close(); + smoosher.close(); + + SmooshedFileMapper mapper = Smoosh.map(tmpDirectory); + + // read from ByteBuffer and check values + CompressedIntsIndexedSupplier supplierFromByteBuffer = CompressedIntsIndexedSupplier.fromByteBuffer( + mapper.mapFile("test"), + byteOrder + ); + IndexedInts indexedInts = supplierFromByteBuffer.get(); + assertEquals(vals.length, indexedInts.size()); + for (int i = 0; i < vals.length; ++i) { + assertEquals(vals[i], indexedInts.get(i)); } + CloseQuietly.close(indexedInts); + mapper.close(); } @Test diff --git a/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java index 3583706973d..625b0c2f818 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java @@ -20,10 +20,10 @@ package io.druid.segment.data; import com.google.common.base.Supplier; -import com.google.common.io.ByteSink; import com.google.common.primitives.Longs; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.Assert; import org.junit.Test; @@ -32,7 +32,6 @@ 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; @@ -51,7 +50,7 @@ public class CompressedLongsSerdeTest { List data = new ArrayList<>(); for (CompressionFactory.LongEncodingStrategy encodingStrategy: CompressionFactory.LongEncodingStrategy.values()) { - for (CompressedObjectStrategy.CompressionStrategy strategy : CompressedObjectStrategy.CompressionStrategy.values()) { + for (CompressionStrategy strategy : CompressionStrategy.values()) { data.add(new Object[]{encodingStrategy, strategy, ByteOrder.BIG_ENDIAN}); data.add(new Object[]{encodingStrategy, strategy, ByteOrder.LITTLE_ENDIAN}); } @@ -60,7 +59,7 @@ public class CompressedLongsSerdeTest } protected final CompressionFactory.LongEncodingStrategy encodingStrategy; - protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + protected final CompressionStrategy compressionStrategy; protected final ByteOrder order; private final long values0[] = {}; @@ -89,7 +88,7 @@ public class CompressedLongsSerdeTest public CompressedLongsSerdeTest( CompressionFactory.LongEncodingStrategy encodingStrategy, - CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressionStrategy compressionStrategy, ByteOrder order ) { @@ -130,8 +129,12 @@ public class CompressedLongsSerdeTest public void testValues(long[] values) throws Exception { - LongSupplierSerializer serializer = CompressionFactory.getLongSerializer(new IOPeonForTesting(), "test", order, - encodingStrategy, compressionStrategy + LongSupplierSerializer serializer = CompressionFactory.getLongSerializer( + new OffHeapMemorySegmentWriteOutMedium(), + "test", + order, + encodingStrategy, + compressionStrategy ); serializer.open(); @@ -141,19 +144,10 @@ public class CompressedLongsSerdeTest Assert.assertEquals(values.length, serializer.size()); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - serializer.closeAndConsolidate( - new ByteSink() - { - @Override - public OutputStream openStream() throws IOException - { - return baos; - } - } - ); + serializer.writeTo(Channels.newChannel(baos), null); Assert.assertEquals(baos.size(), serializer.getSerializedSize()); CompressedLongsIndexedSupplier supplier = CompressedLongsIndexedSupplier - .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order, null); + .fromByteBuffer(ByteBuffer.wrap(baos.toByteArray()), order); IndexedLongs longs = supplier.get(); assertIndexMatchesVals(longs, values); @@ -203,14 +197,13 @@ public class CompressedLongsSerdeTest private void testSupplierSerde(CompressedLongsIndexedSupplier supplier, long[] vals) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); - supplier.writeToChannel(Channels.newChannel(baos)); + supplier.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); Assert.assertEquals(supplier.getSerializedSize(), bytes.length); CompressedLongsIndexedSupplier anotherSupplier = CompressedLongsIndexedSupplier.fromByteBuffer( ByteBuffer.wrap(bytes), - order, - null + order ); IndexedLongs indexed = anotherSupplier.get(); assertIndexMatchesVals(indexed, vals); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java index bd380cb110e..9a98be6c10e 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java @@ -21,6 +21,7 @@ package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; +import io.druid.java.util.common.io.Closer; import io.druid.segment.CompressedVSizeIndexedSupplier; import org.junit.After; import org.junit.Assert; @@ -40,6 +41,7 @@ import java.util.List; */ public class CompressedVSizeIndexedSupplierTest { + private Closer closer; protected List vals; protected WritableSupplier> indexedSupplier; @@ -47,6 +49,7 @@ public class CompressedVSizeIndexedSupplierTest @Before public void setUpSimple() { + closer = Closer.create(); vals = Arrays.asList( new int[1], new int[]{1, 2, 3, 4, 5}, @@ -65,16 +68,20 @@ public class CompressedVSizeIndexedSupplierTest return VSizeIndexedInts.fromArray(input, 20); } } - ), 20, ByteOrder.nativeOrder(), - CompressedObjectStrategy.CompressionStrategy.LZ4 + ), + 20, + ByteOrder.nativeOrder(), + CompressionStrategy.LZ4, + closer ); } @After - public void teardown() + public void teardown() throws IOException { indexedSupplier = null; vals = null; + closer.close(); } @Test @@ -87,7 +94,7 @@ public class CompressedVSizeIndexedSupplierTest public void testSerde() throws IOException { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - indexedSupplier.writeToChannel(Channels.newChannel(baos)); + indexedSupplier.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); Assert.assertEquals(indexedSupplier.getSerializedSize(), bytes.length); @@ -139,8 +146,7 @@ public class CompressedVSizeIndexedSupplierTest { return CompressedVSizeIndexedSupplier.fromByteBuffer( buffer, - ByteOrder.nativeOrder(), - null + ByteOrder.nativeOrder() ); } } 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 cf52bab2054..b3f86aef921 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -32,9 +32,11 @@ import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; import io.druid.segment.CompressedVSizeIndexedV3Supplier; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; -import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -45,8 +47,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; import java.nio.file.Files; import java.util.ArrayList; import java.util.List; @@ -65,14 +65,13 @@ public class CompressedVSizeIndexedV3WriterTest CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER }; private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; - private final IOPeon ioPeon = new TmpFileIOPeon(); - private final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + private final CompressionStrategy compressionStrategy; private final ByteOrder byteOrder; private final Random rand = new Random(0); private List vals; public CompressedVSizeIndexedV3WriterTest( - CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressionStrategy compressionStrategy, ByteOrder byteOrder ) { @@ -84,7 +83,7 @@ public class CompressedVSizeIndexedV3WriterTest public static Iterable compressionStrategiesAndByteOrders() { Set> combinations = Sets.cartesianProduct( - Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), + Sets.newHashSet(CompressionStrategy.noNoneValues()), Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) ); @@ -116,59 +115,51 @@ public class CompressedVSizeIndexedV3WriterTest private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFactor) throws Exception { FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); - final IndexedMultivalue indexedMultivalue; - try (IOPeon ioPeon = new TmpFileIOPeon()) { + try (SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium()) { int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( - ioPeon, "offset", offsetChunkFactor, byteOrder, compressionStrategy + segmentWriteOutMedium, "offset", offsetChunkFactor, byteOrder, compressionStrategy ); CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter( - ioPeon, "value", maxValue, valueChunkFactor, byteOrder, compressionStrategy + segmentWriteOutMedium, "value", maxValue, valueChunkFactor, byteOrder, compressionStrategy ); CompressedVSizeIndexedV3Writer writer = new CompressedVSizeIndexedV3Writer(offsetWriter, valueWriter); CompressedVSizeIndexedV3Supplier supplierFromIterable = CompressedVSizeIndexedV3Supplier.fromIterable( - Iterables.transform( - vals, new Function() - { - @Nullable - @Override - public IndexedInts apply(@Nullable final int[] input) - { - return ArrayBasedIndexedInts.of(input); - } - } - ), offsetChunkFactor, maxValue, byteOrder, compressionStrategy + Iterables.transform(vals, ArrayBasedIndexedInts::of), + offsetChunkFactor, + maxValue, + byteOrder, + compressionStrategy, + segmentWriteOutMedium.getCloser() ); writer.open(); for (int[] val : vals) { writer.add(val); } - writer.close(); long writtenLength = writer.getSerializedSize(); - final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); - writer.writeToChannel(outputChannel, smoosher); - outputChannel.close(); + final WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + writer.writeTo(writeOutBytes, smoosher); smoosher.close(); assertEquals(writtenLength, supplierFromIterable.getSerializedSize()); // read from ByteBuffer and check values CompressedVSizeIndexedV3Supplier supplierFromByteBuffer = CompressedVSizeIndexedV3Supplier.fromByteBuffer( - ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), - byteOrder, - null + ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())), + byteOrder ); - indexedMultivalue = supplierFromByteBuffer.get(); - assertEquals(indexedMultivalue.size(), vals.size()); - for (int i = 0; i < vals.size(); ++i) { - IndexedInts subVals = indexedMultivalue.get(i); - assertEquals(subVals.size(), vals.get(i).length); - for (int j = 0; j < subVals.size(); ++j) { - assertEquals(subVals.get(j), vals.get(i)[j]); + + try (final IndexedMultivalue indexedMultivalue = supplierFromByteBuffer.get()) { + assertEquals(indexedMultivalue.size(), vals.size()); + for (int i = 0; i < vals.size(); ++i) { + IndexedInts subVals = indexedMultivalue.get(i); + assertEquals(subVals.size(), vals.get(i).length); + for (int j = 0; j < subVals.size(); ++j) { + assertEquals(subVals.get(j), vals.get(i)[j]); + } } } - CloseQuietly.close(indexedMultivalue); } } @@ -195,12 +186,6 @@ public class CompressedVSizeIndexedV3WriterTest vals = null; } - @After - public void tearDown() throws Exception - { - ioPeon.close(); - } - @Test public void testSmallData() throws Exception { @@ -244,33 +229,28 @@ public class CompressedVSizeIndexedV3WriterTest FileSmoosher smoosher = new FileSmoosher(tmpDirectory); int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; - try (IOPeon ioPeon = new TmpFileIOPeon()) { + try (SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium()) { CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( + segmentWriteOutMedium, offsetChunkFactor, + byteOrder, compressionStrategy, - new GenericIndexedWriter<>( - ioPeon, "offset", - CompressedIntBufferObjectStrategy.getBufferForOrder( - byteOrder, - compressionStrategy, - offsetChunkFactor - ), + GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, + "offset", + compressionStrategy, Longs.BYTES * 250000 ) ); - GenericIndexedWriter genericIndexed = new GenericIndexedWriter<>( - ioPeon, + GenericIndexedWriter genericIndexed = GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, "value", - CompressedByteBufferObjectStrategy.getBufferForOrder( - byteOrder, - compressionStrategy, - valueChunkFactor * VSizeIndexedInts.getNumBytesForMax(maxValue) - + CompressedVSizeIntsIndexedSupplier.bufferPadding(VSizeIndexedInts.getNumBytesForMax(maxValue)) - ), + compressionStrategy, Longs.BYTES * 250000 ); CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter( + segmentWriteOutMedium, maxValue, valueChunkFactor, byteOrder, @@ -282,21 +262,16 @@ public class CompressedVSizeIndexedV3WriterTest for (int[] val : vals) { writer.add(val); } - writer.close(); - final SmooshedWriter channel = smoosher.addWithSmooshedWriter( - "test", - writer.getSerializedSize() - ); - writer.writeToChannel(channel, smoosher); + final SmooshedWriter channel = smoosher.addWithSmooshedWriter("test", writer.getSerializedSize()); + writer.writeTo(channel, smoosher); channel.close(); smoosher.close(); SmooshedFileMapper mapper = Smoosh.map(tmpDirectory); CompressedVSizeIndexedV3Supplier supplierFromByteBuffer = CompressedVSizeIndexedV3Supplier.fromByteBuffer( mapper.mapFile("test"), - byteOrder, - mapper + byteOrder ); IndexedMultivalue indexedMultivalue = supplierFromByteBuffer.get(); assertEquals(indexedMultivalue.size(), vals.size()); diff --git a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java index d33d7e48383..c22a58f9e0f 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplierTest.java @@ -26,7 +26,9 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; +import io.druid.java.util.common.io.Closer; import io.druid.segment.CompressedPools; +import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.After; import org.junit.Assert; @@ -54,14 +56,14 @@ public class CompressedVSizeIntsIndexedSupplierTest extends CompressionStrategyT @Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}") public static Iterable compressionStrategies() { - final Iterable compressionStrategies = Iterables.transform( + final Iterable compressionStrategies = Iterables.transform( CompressionStrategyTest.compressionStrategies(), - new Function() + new Function() { @Override - public CompressedObjectStrategy.CompressionStrategy apply(Object[] input) + public CompressionStrategy apply(Object[] input) { - return (CompressedObjectStrategy.CompressionStrategy) input[0]; + return (CompressionStrategy) input[0]; } } ); @@ -85,12 +87,13 @@ public class CompressedVSizeIntsIndexedSupplierTest extends CompressionStrategyT private static final int[] MAX_VALUES = new int[] {0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; - public CompressedVSizeIntsIndexedSupplierTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy, ByteOrder byteOrder) + public CompressedVSizeIntsIndexedSupplierTest(CompressionStrategy compressionStrategy, ByteOrder byteOrder) { super(compressionStrategy); this.byteOrder = byteOrder; } + private Closer closer; private IndexedInts indexed; private CompressedVSizeIntsIndexedSupplier supplier; private int[] vals; @@ -100,6 +103,7 @@ public class CompressedVSizeIntsIndexedSupplierTest extends CompressionStrategyT @Before public void setUp() throws Exception { + closer = Closer.create(); CloseQuietly.close(indexed); indexed = null; supplier = null; @@ -110,6 +114,7 @@ public class CompressedVSizeIntsIndexedSupplierTest extends CompressionStrategyT public void tearDown() throws Exception { CloseQuietly.close(indexed); + closer.close(); } private void setupSimple(final int chunkSize) @@ -119,11 +124,12 @@ public class CompressedVSizeIntsIndexedSupplierTest extends CompressionStrategyT vals = new int[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 16}; supplier = CompressedVSizeIntsIndexedSupplier.fromList( - Ints.asList(vals), + IntArrayList.wrap(vals), Ints.max(vals), chunkSize, ByteOrder.nativeOrder(), - compressionStrategy + compressionStrategy, + closer ); indexed = supplier.get(); @@ -142,14 +148,14 @@ public class CompressedVSizeIntsIndexedSupplierTest extends CompressionStrategyT ByteArrayOutputStream baos = new ByteArrayOutputStream(); final CompressedVSizeIntsIndexedSupplier theSupplier = CompressedVSizeIntsIndexedSupplier.fromList( - Ints.asList(vals), Ints.max(vals), chunkSize, byteOrder, compressionStrategy + IntArrayList.wrap(vals), Ints.max(vals), chunkSize, byteOrder, compressionStrategy, closer ); - theSupplier.writeToChannel(Channels.newChannel(baos)); + theSupplier.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); Assert.assertEquals(theSupplier.getSerializedSize(), bytes.length); - supplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), byteOrder, null); + supplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(ByteBuffer.wrap(bytes), byteOrder); indexed = supplier.get(); } 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 54c1fdb25ea..ffba7eaddaf 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java @@ -29,6 +29,10 @@ import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; +import it.unimi.dsi.fastutil.ints.IntArrayList; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; import org.junit.After; @@ -41,8 +45,6 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; import java.util.List; import java.util.Random; import java.util.Set; @@ -53,13 +55,13 @@ import static org.junit.Assert.assertEquals; public class CompressedVSizeIntsIndexedWriterTest { private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; - private final IOPeon ioPeon = new TmpFileIOPeon(); - private final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + private final SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); + private final CompressionStrategy compressionStrategy; private final ByteOrder byteOrder; private final Random rand = new Random(0); private int[] vals; public CompressedVSizeIntsIndexedWriterTest( - CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressionStrategy compressionStrategy, ByteOrder byteOrder ) { @@ -71,7 +73,7 @@ public class CompressedVSizeIntsIndexedWriterTest public static Iterable compressionStrategiesAndByteOrders() { Set> combinations = Sets.cartesianProduct( - Sets.newHashSet(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), + Sets.newHashSet(CompressionStrategy.noNoneValues()), Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) ); @@ -96,7 +98,7 @@ public class CompressedVSizeIntsIndexedWriterTest @After public void tearDown() throws Exception { - ioPeon.close(); + segmentWriteOutMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -112,29 +114,36 @@ public class CompressedVSizeIntsIndexedWriterTest FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter( - ioPeon, "test", vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy + segmentWriteOutMedium, + "test", + vals.length > 0 ? Ints.max(vals) : 0, + chunkSize, + byteOrder, + compressionStrategy ); CompressedVSizeIntsIndexedSupplier supplierFromList = CompressedVSizeIntsIndexedSupplier.fromList( - Ints.asList(vals), vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy + IntArrayList.wrap(vals), + vals.length > 0 ? Ints.max(vals) : 0, + chunkSize, + byteOrder, + compressionStrategy, + segmentWriteOutMedium.getCloser() ); writer.open(); for (int val : vals) { writer.add(val); } - writer.close(); long writtenLength = writer.getSerializedSize(); - final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); - writer.writeToChannel(outputChannel, smoosher); - outputChannel.close(); + final WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + writer.writeTo(writeOutBytes, smoosher); smoosher.close(); assertEquals(writtenLength, supplierFromList.getSerializedSize()); // read from ByteBuffer and check values CompressedVSizeIntsIndexedSupplier supplierFromByteBuffer = CompressedVSizeIntsIndexedSupplier.fromByteBuffer( - ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), - byteOrder, - null + ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())), + byteOrder ); IndexedInts indexedInts = supplierFromByteBuffer.get(); for (int i = 0; i < vals.length; ++i) { @@ -177,20 +186,18 @@ public class CompressedVSizeIntsIndexedWriterTest File tmpDirectory = FileUtils.getTempDirectory(); FileSmoosher smoosher = new FileSmoosher(tmpDirectory); - int maxValue = vals.length > 0 ? Ints.max(vals) : 0; - GenericIndexedWriter genericIndexed = new GenericIndexedWriter<>( - ioPeon, + GenericIndexedWriter genericIndexed = GenericIndexedWriter.ofCompressedByteBuffers( + segmentWriteOutMedium, "test", - CompressedByteBufferObjectStrategy.getBufferForOrder( - byteOrder, - compressionStrategy, - chunkSize * VSizeIndexedInts.getNumBytesForMax(maxValue) - + CompressedVSizeIntsIndexedSupplier.bufferPadding(VSizeIndexedInts.getNumBytesForMax(maxValue)) - ), + compressionStrategy, Longs.BYTES * 10000 ); CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter( - vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy, + segmentWriteOutMedium, + vals.length > 0 ? Ints.max(vals) : 0, + chunkSize, + byteOrder, + compressionStrategy, genericIndexed ); writer.open(); @@ -198,12 +205,11 @@ public class CompressedVSizeIntsIndexedWriterTest writer.add(val); } - writer.close(); final SmooshedWriter channel = smoosher.addWithSmooshedWriter( "test", writer.getSerializedSize() ); - writer.writeToChannel(channel, smoosher); + writer.writeTo(channel, smoosher); channel.close(); smoosher.close(); @@ -211,8 +217,7 @@ public class CompressedVSizeIntsIndexedWriterTest CompressedVSizeIntsIndexedSupplier supplierFromByteBuffer = CompressedVSizeIntsIndexedSupplier.fromByteBuffer( mapper.mapFile("test"), - byteOrder, - mapper + byteOrder ); IndexedInts indexedInts = supplierFromByteBuffer.get(); 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 4867cd27689..e94d53f2f7a 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java @@ -21,26 +21,24 @@ package io.druid.segment.data; import com.google.common.base.Function; import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; +import io.druid.java.util.common.io.Closer; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.List; import java.util.Random; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.Callable; -import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -52,11 +50,11 @@ public class CompressionStrategyTest public static Iterable compressionStrategies() { return Iterables.transform( - Arrays.asList(CompressedObjectStrategy.CompressionStrategy.noNoneValues()), - new Function() + Arrays.asList(CompressionStrategy.noNoneValues()), + new Function() { @Override - public Object[] apply(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + public Object[] apply(CompressionStrategy compressionStrategy) { return new Object[]{compressionStrategy}; } @@ -64,9 +62,9 @@ public class CompressionStrategyTest ); } - protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + protected final CompressionStrategy compressionStrategy; - public CompressionStrategyTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + public CompressionStrategyTest(CompressionStrategy compressionStrategy) { this.compressionStrategy = compressionStrategy; } @@ -83,25 +81,27 @@ public class CompressionStrategyTest random.nextBytes(originalData); } + private Closer closer; + + @Before + public void createCloser() + { + closer = Closer.create(); + } + + @After + public void closeCloser() throws IOException + { + closer.close(); + } + @Test public void testBasicOperations() { - ByteBuffer compressed = ByteBuffer.wrap(compressionStrategy.getCompressor().compress(originalData)); + ByteBuffer compressionOut = compressionStrategy.getCompressor().allocateOutBuffer(originalData.length, closer); + ByteBuffer compressed = compressionStrategy.getCompressor().compress(ByteBuffer.wrap(originalData), compressionOut); ByteBuffer output = ByteBuffer.allocate(originalData.length); - compressionStrategy.getDecompressor().decompress(compressed, compressed.array().length, output); - byte[] checkArray = new byte[DATA_SIZER]; - output.get(checkArray); - Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); - } - - - @Test - public void testOutputSizeKnownOperations() - { - ByteBuffer compressed = ByteBuffer.wrap(compressionStrategy.getCompressor().compress(originalData)); - ByteBuffer output = ByteBuffer.allocate(originalData.length); - compressionStrategy.getDecompressor() - .decompress(compressed, compressed.array().length, output, originalData.length); + compressionStrategy.getDecompressor().decompress(compressed, compressed.remaining(), output); byte[] checkArray = new byte[DATA_SIZER]; output.get(checkArray); Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); @@ -110,9 +110,10 @@ public class CompressionStrategyTest @Test public void testDirectMemoryOperations() { - ByteBuffer compressed = ByteBuffer.wrap(compressionStrategy.getCompressor().compress(originalData)); + ByteBuffer compressionOut = compressionStrategy.getCompressor().allocateOutBuffer(originalData.length, closer); + ByteBuffer compressed = compressionStrategy.getCompressor().compress(ByteBuffer.wrap(originalData), compressionOut); ByteBuffer output = ByteBuffer.allocateDirect(originalData.length); - compressionStrategy.getDecompressor().decompress(compressed, compressed.array().length, output); + compressionStrategy.getDecompressor().decompress(compressed, compressed.remaining(), output); byte[] checkArray = new byte[DATA_SIZER]; output.get(checkArray); Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); @@ -139,9 +140,10 @@ public class CompressionStrategyTest @Override public Boolean call() throws Exception { - ByteBuffer compressed = ByteBuffer.wrap(compressionStrategy.getCompressor().compress(originalData)); + ByteBuffer compressionOut = compressionStrategy.getCompressor().allocateOutBuffer(originalData.length, closer); + ByteBuffer compressed = compressionStrategy.getCompressor().compress(ByteBuffer.wrap(originalData), compressionOut); ByteBuffer output = ByteBuffer.allocate(originalData.length); - compressionStrategy.getDecompressor().decompress(compressed, compressed.array().length, output); + compressionStrategy.getDecompressor().decompress(compressed, compressed.remaining(), output); byte[] checkArray = new byte[DATA_SIZER]; output.get(checkArray); Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); @@ -156,36 +158,4 @@ public class CompressionStrategyTest Assert.assertTrue((Boolean) result.get()); } } - - - @Test(timeout = 60000) - public void testKnownSizeConcurrency() throws Exception - { - final int numThreads = 20; - - ListeningExecutorService threadPoolExecutor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(numThreads)); - List> results = new ArrayList<>(); - for (int i = 0; i < numThreads; ++i) { - results.add( - threadPoolExecutor.submit( - new Runnable() - { - @Override - public void run() - { - ByteBuffer compressed = ByteBuffer.wrap(compressionStrategy.getCompressor().compress(originalData)); - ByteBuffer output = ByteBuffer.allocate(originalData.length); - // TODO: Lambdas would be nice here whenever we use Java 8 - compressionStrategy.getDecompressor() - .decompress(compressed, compressed.array().length, output, originalData.length); - byte[] checkArray = new byte[DATA_SIZER]; - output.get(checkArray); - Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); - } - } - ) - ); - } - Futures.allAsList(results).get(); - } } diff --git a/processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java b/processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java new file mode 100644 index 00000000000..ec7f5d8e69b --- /dev/null +++ b/processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java @@ -0,0 +1,59 @@ +/* + * 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 io.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; + +public class GenericIndexedStringWriterTest +{ + @Test + public void testRandomAccess() throws IOException + { + OnHeapMemorySegmentWriteOutMedium segmentWriteOutMedium = new OnHeapMemorySegmentWriteOutMedium(); + GenericIndexedWriter writer = new GenericIndexedWriter<>( + segmentWriteOutMedium, + "test", + GenericIndexed.STRING_STRATEGY + ); + writer.open(); + writer.write(null); + List strings = new ArrayList<>(); + strings.add(null); + ThreadLocalRandom r = ThreadLocalRandom.current(); + for (int i = 0; i < 100_000; i++) { + byte[] bs = new byte[r.nextInt(1, 10)]; + r.nextBytes(bs); + String s = new String(bs, StandardCharsets.US_ASCII); + strings.add(s); + writer.write(s); + } + for (int i = 0; i < strings.size(); i++) { + Assert.assertEquals(strings.get(i), writer.get(i)); + } + } +} diff --git a/processing/src/test/java/io/druid/segment/data/GenericIndexedTest.java b/processing/src/test/java/io/druid/segment/data/GenericIndexedTest.java index f4bfd70bfff..2671b547977 100644 --- a/processing/src/test/java/io/druid/segment/data/GenericIndexedTest.java +++ b/processing/src/test/java/io/druid/segment/data/GenericIndexedTest.java @@ -125,7 +125,7 @@ public class GenericIndexedTest { ByteArrayOutputStream baos = new ByteArrayOutputStream(); final WritableByteChannel channel = Channels.newChannel(baos); - indexed.writeToChannel(channel); + indexed.writeTo(channel, null); channel.close(); final ByteBuffer byteBuffer = ByteBuffer.wrap(baos.toByteArray()); diff --git a/processing/src/test/java/io/druid/segment/data/IOPeonForTesting.java b/processing/src/test/java/io/druid/segment/data/IOPeonForTesting.java deleted file mode 100644 index e4043e776aa..00000000000 --- a/processing/src/test/java/io/druid/segment/data/IOPeonForTesting.java +++ /dev/null @@ -1,76 +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.collect.Maps; -import io.druid.java.util.common.StringUtils; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Map; - -/** - */ -class IOPeonForTesting implements IOPeon -{ - Map outStreams = Maps.newHashMap(); - - @Override - public OutputStream makeOutputStream(String filename) throws IOException - { - ByteArrayOutputStream stream = outStreams.get(filename); - - if (stream == null) { - stream = new ByteArrayOutputStream(); - outStreams.put(filename, stream); - } - - return stream; - } - - @Override - public InputStream makeInputStream(String filename) throws IOException - { - ByteArrayOutputStream outStream = outStreams.get(filename); - - if (outStream == null) { - throw new FileNotFoundException(StringUtils.format("unknown file[%s]", filename)); - } - - return new ByteArrayInputStream(outStream.toByteArray()); - } - - @Override - public void close() throws IOException - { - outStreams.clear(); - } - - @Override - public File getFile(String filename) - { - return null; - } -} diff --git a/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsTest.java b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsTest.java index c8ebcf1290b..9bd87cdba2e 100644 --- a/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsTest.java @@ -19,14 +19,12 @@ package io.druid.segment.data; -import com.google.common.primitives.Ints; import org.junit.Assert; import org.junit.Test; import java.io.ByteArrayOutputStream; import java.nio.ByteBuffer; import java.nio.channels.Channels; -import java.util.List; /** */ @@ -52,7 +50,7 @@ public class VSizeIndexedIntsTest VSizeIndexedInts ints = VSizeIndexedInts.fromArray(array); ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ints.writeToChannel(Channels.newChannel(baos)); + ints.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); Assert.assertEquals(ints.getSerializedSize(), bytes.length); @@ -64,16 +62,4 @@ public class VSizeIndexedIntsTest Assert.assertEquals(array[i], deserialized.get(i)); } } - - @Test - public void testGetBytesNoPaddingfromList() throws Exception - { - final int[] array = {1, 2, 4, 5, 6, 8, 9, 10}; - List list = Ints.asList(array); - int maxValue = Ints.max(array); - VSizeIndexedInts ints = VSizeIndexedInts.fromList(list, maxValue); - byte[] bytes1 = ints.getBytesNoPadding(); - byte[] bytes2 = VSizeIndexedInts.getBytesNoPaddingFromList(list, maxValue); - Assert.assertArrayEquals(bytes1, bytes2); - } } diff --git a/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java index 4a0a4a2926a..2649b4ca8ea 100644 --- a/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java @@ -20,6 +20,10 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import it.unimi.dsi.fastutil.ints.IntArrayList; import org.apache.commons.io.IOUtils; import org.junit.After; import org.junit.Before; @@ -27,8 +31,6 @@ import org.junit.Test; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; import java.util.Random; import static org.junit.Assert.assertEquals; @@ -37,7 +39,7 @@ public class VSizeIndexedIntsWriterTest { private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; - private final IOPeon ioPeon = new TmpFileIOPeon(); + private final SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); private final Random rand = new Random(0); private int[] vals; @@ -50,7 +52,7 @@ public class VSizeIndexedIntsWriterTest @After public void tearDown() throws Exception { - ioPeon.close(); + segmentWriteOutMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -64,28 +66,24 @@ public class VSizeIndexedIntsWriterTest private void checkSerializedSizeAndData() throws Exception { int maxValue = vals.length == 0 ? 0 : Ints.max(vals); - VSizeIndexedIntsWriter writer = new VSizeIndexedIntsWriter( - ioPeon, "test", maxValue - ); + VSizeIndexedIntsWriter writer = new VSizeIndexedIntsWriter(segmentWriteOutMedium, maxValue); VSizeIndexedInts intsFromList = VSizeIndexedInts.fromList( - Ints.asList(vals), maxValue + IntArrayList.wrap(vals), maxValue ); writer.open(); for (int val : vals) { writer.add(val); } - writer.close(); long writtenLength = writer.getSerializedSize(); - final WritableByteChannel outputChannel = Channels.newChannel(ioPeon.makeOutputStream("output")); - writer.writeToChannel(outputChannel, null); - outputChannel.close(); + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + writer.writeTo(writeOutBytes, null); assertEquals(writtenLength, intsFromList.getSerializedSize()); // read from ByteBuffer and check values VSizeIndexedInts intsFromByteBuffer = VSizeIndexedInts.readFromByteBuffer( - ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))) + ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())) ); assertEquals(vals.length, intsFromByteBuffer.size()); for (int i = 0; i < vals.length; ++i) { diff --git a/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java b/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java index 215a4d12f05..bb366937ece 100644 --- a/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedTest.java @@ -60,7 +60,7 @@ public class VSizeIndexedTest assertSame(someInts, indexed); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); - indexed.writeToChannel(Channels.newChannel(baos)); + indexed.writeTo(Channels.newChannel(baos), null); final byte[] bytes = baos.toByteArray(); Assert.assertEquals(indexed.getSerializedSize(), bytes.length); 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 34982f8366e..f38c49d7e6f 100644 --- a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java @@ -27,13 +27,16 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import io.druid.common.guava.SettableSupplier; -import io.druid.java.util.common.Intervals; import io.druid.data.input.InputRow; +import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.BitmapResultFactory; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.CountAggregatorFactory; @@ -50,12 +53,10 @@ import io.druid.segment.ColumnSelectorFactory; import io.druid.segment.Cursor; import io.druid.segment.DimensionSelector; import io.druid.segment.IndexBuilder; -import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexStorageAdapter; import io.druid.segment.StorageAdapter; -import io.druid.segment.TestHelper; import io.druid.segment.VirtualColumn; import io.druid.segment.VirtualColumns; import io.druid.segment.column.ValueType; @@ -181,8 +182,9 @@ public abstract class BaseFilterTest "roaring", new RoaringBitmapSerdeFactory(true) ); - final Map indexMergers = ImmutableMap.of( - "IndexMergerV9", TestHelper.getTestIndexMergerV9() + final Map segmentWriteOutMediumFactories = ImmutableMap.of( + "tmpFile segment write-out medium", TmpFileSegmentWriteOutMediumFactory.instance(), + "off-heap memory segment write-out medium", OffHeapMemorySegmentWriteOutMediumFactory.instance() ); final Map>> finishers = ImmutableMap.of( @@ -246,25 +248,23 @@ public abstract class BaseFilterTest ); for (Map.Entry bitmapSerdeFactoryEntry : bitmapSerdeFactories.entrySet()) { - for (Map.Entry indexMergerEntry : indexMergers.entrySet()) { - for (Map.Entry>> finisherEntry : finishers.entrySet()) { + for (Map.Entry segmentWriteOutMediumFactoryEntry : + segmentWriteOutMediumFactories.entrySet()) { + for (Map.Entry>> finisherEntry : + finishers.entrySet()) { for (boolean cnf : ImmutableList.of(false, true)) { for (boolean optimize : ImmutableList.of(false, true)) { final String testName = StringUtils.format( "bitmaps[%s], indexMerger[%s], finisher[%s], optimize[%s]", bitmapSerdeFactoryEntry.getKey(), - indexMergerEntry.getKey(), + segmentWriteOutMediumFactoryEntry.getKey(), finisherEntry.getKey(), optimize ); - final IndexBuilder indexBuilder = IndexBuilder.create() - .indexSpec(new IndexSpec( - bitmapSerdeFactoryEntry.getValue(), - null, - null, - null - )) - .indexMerger(indexMergerEntry.getValue()); + final IndexBuilder indexBuilder = IndexBuilder + .create() + .indexSpec(new IndexSpec(bitmapSerdeFactoryEntry.getValue(), null, null, null)) + .segmentWriteOutMediumFactory(segmentWriteOutMediumFactoryEntry.getValue()); constructors.add(new Object[]{testName, indexBuilder, finisherEntry.getValue(), cnf, optimize}); } diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java index f2def054670..3ff7337942c 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -32,6 +32,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -65,6 +66,7 @@ import org.junit.runners.Parameterized; import java.io.File; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; @@ -84,8 +86,6 @@ public class SpatialFilterBonusTest new LongSumAggregatorFactory("val", "val") }; private static List DIMS = Lists.newArrayList("dim", "dim.geo"); - private static final IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); - private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(); private final Segment segment; @@ -97,23 +97,19 @@ public class SpatialFilterBonusTest @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { - final IndexSpec indexSpec = new IndexSpec(); - final IncrementalIndex rtIndex = makeIncrementalIndex(); - final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec); - final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec); - return Arrays.asList( - new Object[][]{ - { - new IncrementalIndexSegment(rtIndex, null) - }, - { - new QueryableIndexSegment(null, mMappedTestIndex) - }, - { - new QueryableIndexSegment(null, mergedRealtimeIndex) - } - } - ); + List argumentArrays = new ArrayList<>(); + for (SegmentWriteOutMediumFactory segmentWriteOutMediumFactory : SegmentWriteOutMediumFactory.builtInFactories()) { + IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + IndexIO indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); + final IndexSpec indexSpec = new IndexSpec(); + final IncrementalIndex rtIndex = makeIncrementalIndex(); + final QueryableIndex mMappedTestIndex = makeQueryableIndex(indexSpec, indexMerger, indexIO); + final QueryableIndex mergedRealtimeIndex = makeMergedQueryableIndex(indexSpec, indexMerger, indexIO); + argumentArrays.add(new Object[] {new IncrementalIndexSegment(rtIndex, null)}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mMappedTestIndex)}); + argumentArrays.add(new Object[] {new QueryableIndexSegment(null, mergedRealtimeIndex)}); + } + return argumentArrays; } private static IncrementalIndex makeIncrementalIndex() throws IOException @@ -246,7 +242,8 @@ public class SpatialFilterBonusTest return theIndex; } - private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOException + private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec, IndexMerger indexMerger, IndexIO indexIO) + throws IOException { IncrementalIndex theIndex = makeIncrementalIndex(); File tmpFile = File.createTempFile("billy", "yay"); @@ -254,11 +251,15 @@ public class SpatialFilterBonusTest tmpFile.mkdirs(); tmpFile.deleteOnExit(); - INDEX_MERGER.persist(theIndex, tmpFile, indexSpec); - return INDEX_IO.loadIndex(tmpFile); + indexMerger.persist(theIndex, tmpFile, indexSpec, null); + return indexIO.loadIndex(tmpFile); } - private static QueryableIndex makeMergedQueryableIndex(final IndexSpec indexSpec) + private static QueryableIndex makeMergedQueryableIndex( + final IndexSpec indexSpec, + final IndexMerger indexMerger, + final IndexIO indexIO + ) { try { IncrementalIndex first = new IncrementalIndex.Builder() @@ -444,21 +445,22 @@ public class SpatialFilterBonusTest mergedFile.mkdirs(); mergedFile.deleteOnExit(); - INDEX_MERGER.persist(first, DATA_INTERVAL, firstFile, indexSpec); - INDEX_MERGER.persist(second, DATA_INTERVAL, secondFile, indexSpec); - INDEX_MERGER.persist(third, DATA_INTERVAL, thirdFile, indexSpec); + indexMerger.persist(first, DATA_INTERVAL, firstFile, indexSpec, null); + indexMerger.persist(second, DATA_INTERVAL, secondFile, indexSpec, null); + indexMerger.persist(third, DATA_INTERVAL, thirdFile, indexSpec, null); - QueryableIndex mergedRealtime = INDEX_IO.loadIndex( - INDEX_MERGER.mergeQueryableIndex( + QueryableIndex mergedRealtime = indexIO.loadIndex( + indexMerger.mergeQueryableIndex( Arrays.asList( - INDEX_IO.loadIndex(firstFile), - INDEX_IO.loadIndex(secondFile), - INDEX_IO.loadIndex(thirdFile) + indexIO.loadIndex(firstFile), + indexIO.loadIndex(secondFile), + indexIO.loadIndex(thirdFile) ), true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); diff --git a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java index 7e8b1871815..31311cb15f2 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -31,6 +31,7 @@ import io.druid.data.input.impl.SpatialDimensionSchema; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -73,8 +74,8 @@ import java.util.Random; @RunWith(Parameterized.class) public class SpatialFilterTest { - private static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(); - private static IndexIO INDEX_IO = TestHelper.getTestIndexIO(); + private static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); + private static IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()); public static final int NUM_POINTS = 5000; private static Interval DATA_INTERVAL = Intervals.of("2013-01-01/2013-01-07"); @@ -270,7 +271,7 @@ public class SpatialFilterTest tmpFile.mkdirs(); tmpFile.deleteOnExit(); - INDEX_MERGER.persist(theIndex, tmpFile, indexSpec); + INDEX_MERGER.persist(theIndex, tmpFile, indexSpec, null); return INDEX_IO.loadIndex(tmpFile); } @@ -497,9 +498,9 @@ public class SpatialFilterTest mergedFile.mkdirs(); mergedFile.deleteOnExit(); - INDEX_MERGER.persist(first, DATA_INTERVAL, firstFile, indexSpec); - INDEX_MERGER.persist(second, DATA_INTERVAL, secondFile, indexSpec); - INDEX_MERGER.persist(third, DATA_INTERVAL, thirdFile, indexSpec); + INDEX_MERGER.persist(first, DATA_INTERVAL, firstFile, indexSpec, null); + INDEX_MERGER.persist(second, DATA_INTERVAL, secondFile, indexSpec, null); + INDEX_MERGER.persist(third, DATA_INTERVAL, thirdFile, indexSpec, null); QueryableIndex mergedRealtime = INDEX_IO.loadIndex( INDEX_MERGER.mergeQueryableIndex( @@ -507,7 +508,8 @@ public class SpatialFilterTest true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); 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 619a5e3e8ae..0e7a4e91cb3 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java @@ -23,8 +23,8 @@ import io.druid.segment.IndexSpec; import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; import io.druid.segment.data.BitmapValues; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.IncrementalIndexTest; import org.junit.Assert; @@ -37,8 +37,8 @@ public class IncrementalIndexAdapterTest { private static final IndexSpec INDEX_SPEC = new IndexSpec( new ConciseBitmapSerdeFactory(), - CompressedObjectStrategy.CompressionStrategy.LZ4, - CompressedObjectStrategy.CompressionStrategy.LZ4, + CompressionStrategy.LZ4, + CompressionStrategy.LZ4, CompressionFactory.LongEncodingStrategy.LONGS ); diff --git a/processing/src/test/java/io/druid/segment/loading/SegmentizerFactoryTest.java b/processing/src/test/java/io/druid/segment/loading/SegmentizerFactoryTest.java index a05e4e98389..5df84c4ddd8 100644 --- a/processing/src/test/java/io/druid/segment/loading/SegmentizerFactoryTest.java +++ b/processing/src/test/java/io/druid/segment/loading/SegmentizerFactoryTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.SegmentizerModule; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.segment.IndexIO; import io.druid.segment.column.ColumnConfig; import org.junit.Assert; @@ -42,14 +43,18 @@ public class SegmentizerFactoryTest FileOutputStream fos = new FileOutputStream(factoryFile); ObjectMapper mapper = new DefaultObjectMapper(); mapper.registerModule(new SegmentizerModule()); - IndexIO indexIO = new IndexIO(mapper, new ColumnConfig() - { - @Override - public int columnCacheSizeBytes() - { - return 777; - } - }); + IndexIO indexIO = new IndexIO( + mapper, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), + new ColumnConfig() + { + @Override + public int columnCacheSizeBytes() + { + return 777; + } + } + ); mapper.setInjectableValues( new InjectableValues.Std().addValue( IndexIO.class, diff --git a/processing/src/test/java/io/druid/segment/serde/HyperUniquesSerdeForTest.java b/processing/src/test/java/io/druid/segment/serde/HyperUniquesSerdeForTest.java index 3f2df176c16..602868d4849 100644 --- a/processing/src/test/java/io/druid/segment/serde/HyperUniquesSerdeForTest.java +++ b/processing/src/test/java/io/druid/segment/serde/HyperUniquesSerdeForTest.java @@ -24,10 +24,10 @@ import com.google.common.hash.HashFunction; import com.metamx.common.StringUtils; import io.druid.data.input.InputRow; import io.druid.hll.HyperLogLogCollector; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.ObjectStrategy; import java.nio.ByteBuffer; @@ -152,10 +152,10 @@ public class HyperUniquesSerdeForTest extends ComplexMetricSerde } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String metric) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String metric) { return LargeColumnSupportedComplexColumnSerializer.createWithColumnSize( - peon, + segmentWriteOutMedium, metric, this.getObjectStrategy(), Integer.MAX_VALUE diff --git a/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java b/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java index a7e7b98cf42..de710e3c8a1 100644 --- a/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java @@ -27,12 +27,12 @@ import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ComplexColumn; import io.druid.segment.column.ValueType; -import io.druid.segment.data.IOPeon; -import io.druid.segment.data.TmpFileIOPeon; import org.apache.commons.io.FileUtils; import org.junit.Assert; import org.junit.Test; @@ -60,11 +60,11 @@ public class LargeColumnSupportedComplexColumnSerializerTest for (int aCase : cases) { File tmpFile = FileUtils.getTempDirectory(); HyperLogLogCollector baseCollector = HyperLogLogCollector.makeLatestCollector(); - try (IOPeon peon = new TmpFileIOPeon(); + try (SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); FileSmoosher v9Smoosher = new FileSmoosher(tmpFile)) { LargeColumnSupportedComplexColumnSerializer serializer = LargeColumnSupportedComplexColumnSerializer - .createWithColumnSize(peon, "test", serde.getObjectStrategy(), columnSize); + .createWithColumnSize(segmentWriteOutMedium, "test", serde.getObjectStrategy(), columnSize); serializer.open(); for (int i = 0; i < aCase; i++) { @@ -74,13 +74,12 @@ public class LargeColumnSupportedComplexColumnSerializerTest baseCollector.fold(collector); serializer.serialize(collector); } - serializer.close(); try (final SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter( "test", serializer.getSerializedSize() )) { - serializer.writeToChannel(channel, v9Smoosher); + serializer.writeTo(channel, v9Smoosher); } } diff --git a/processing/src/test/java/io/druid/segment/writeout/WriteOutBytesTest.java b/processing/src/test/java/io/druid/segment/writeout/WriteOutBytesTest.java new file mode 100644 index 00000000000..6b62ad30c86 --- /dev/null +++ b/processing/src/test/java/io/druid/segment/writeout/WriteOutBytesTest.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.writeout; + +import com.google.common.io.Files; +import com.google.common.primitives.Ints; +import io.druid.java.util.common.StringUtils; +import org.apache.commons.io.IOUtils; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.nio.BufferUnderflowException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collection; + +@RunWith(Parameterized.class) +public class WriteOutBytesTest +{ + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return Arrays.asList( + new Object[] {new TmpFileSegmentWriteOutMedium(Files.createTempDir())}, + new Object[] {new OffHeapMemorySegmentWriteOutMedium()}, + new Object[] {new OnHeapMemorySegmentWriteOutMedium()} + ); + } + + private final SegmentWriteOutMedium segmentWriteOutMedium; + + public WriteOutBytesTest(SegmentWriteOutMedium segmentWriteOutMedium) + { + this.segmentWriteOutMedium = segmentWriteOutMedium; + } + + @Test + public void testWriteOutBytes() throws IOException + { + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + + writeOutBytes.write('1'); + verifyContents(writeOutBytes, "1"); + + writeOutBytes.writeInt(Ints.fromBytes((byte) '2', (byte) '3', (byte) '4', (byte) '5')); + verifyContents(writeOutBytes, "12345"); + + writeOutBytes.write(new byte[] {'a'}); + verifyContents(writeOutBytes, "12345a"); + + writeOutBytes.write(new byte[] {'a', 'b', 'c'}, 1, 1); + verifyContents(writeOutBytes, "12345ab"); + + ByteBuffer bb = ByteBuffer.wrap(new byte[]{'a', 'b', 'c'}); + bb.position(2); + writeOutBytes.write(bb); + Assert.assertEquals(3, bb.position()); + verifyContents(writeOutBytes, "12345abc"); + } + + private void verifyContents(WriteOutBytes writeOutBytes, String expected) throws IOException + { + Assert.assertEquals(expected, IOUtils.toString(writeOutBytes.asInputStream(), StandardCharsets.US_ASCII)); + ByteBuffer bb = ByteBuffer.allocate((int) writeOutBytes.size()); + writeOutBytes.readFully(0, bb); + bb.flip(); + Assert.assertEquals(expected, StringUtils.fromUtf8(bb)); + } + + @Test + public void testCrossBufferRandomAccess() throws IOException + { + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + for (int i = 0; i < ByteBufferWriteOutBytes.BUFFER_SIZE; i++) { + writeOutBytes.write('0'); + } + writeOutBytes.write('1'); + writeOutBytes.write('2'); + writeOutBytes.write('3'); + ByteBuffer bb = ByteBuffer.allocate(4); + writeOutBytes.readFully(ByteBufferWriteOutBytes.BUFFER_SIZE - 1, bb); + bb.flip(); + Assert.assertEquals("0123", StringUtils.fromUtf8(bb)); + } + + @Test(expected = BufferUnderflowException.class) + public void testReadFullyUnderflow() throws IOException + { + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + writeOutBytes.write('1'); + writeOutBytes.readFully(0, ByteBuffer.allocate(2)); + } + + @Test + public void testReadFullyEmptyAtTheEnd() throws IOException + { + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + writeOutBytes.write('1'); + writeOutBytes.readFully(1, ByteBuffer.allocate(0)); + } +} diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 81ef5e2fdc3..aeef72a0cb7 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -64,6 +64,7 @@ import io.druid.guice.security.EscalatorModule; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.storage.derby.DerbyMetadataStorageDruidModule; +import io.druid.segment.writeout.SegmentWriteOutMediumModule; import io.druid.server.emitter.EmitterModule; import io.druid.server.initialization.AuthenticatorMapperModule; import io.druid.server.initialization.AuthorizerMapperModule; @@ -360,6 +361,7 @@ public class Initialization new AnnouncerModule(), new AWSModule(), new MetricsModule(), + new SegmentWriteOutMediumModule(), new ServerModule(), new DruidProcessingConfigModule(), new StorageNodeModule(), diff --git a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java index 5338055a4d8..32c2f5ec962 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.io.Files; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import io.druid.segment.realtime.appenderator.AppenderatorConfig; import io.druid.segment.realtime.plumber.IntervalStartVersioningPolicy; @@ -33,6 +34,7 @@ import io.druid.timeline.partition.NoneShardSpec; import io.druid.timeline.partition.ShardSpec; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.File; /** @@ -57,7 +59,7 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig } // Might make sense for this to be a builder - public static RealtimeTuningConfig makeDefaultTuningConfig(final File basePersistDirectory) + public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File basePersistDirectory) { return new RealtimeTuningConfig( defaultMaxRowsInMemory, @@ -74,7 +76,8 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig 0, defaultReportParseExceptions, defaultHandoffConditionTimeout, - defaultAlertTimeout + defaultAlertTimeout, + null ); } @@ -92,6 +95,8 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig private final boolean reportParseExceptions; private final long handoffConditionTimeout; private final long alertTimeout; + @Nullable + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public RealtimeTuningConfig( @@ -110,7 +115,8 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig @JsonProperty("mergeThreadPriority") int mergeThreadPriority, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, - @JsonProperty("alertTimeout") Long alertTimeout + @JsonProperty("alertTimeout") Long alertTimeout, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; @@ -138,6 +144,7 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig this.alertTimeout = alertTimeout == null ? defaultAlertTimeout : alertTimeout; Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Override @@ -240,6 +247,14 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig return alertTimeout; } + @Override + @JsonProperty + @Nullable + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() + { + return segmentWriteOutMediumFactory; + } + public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( @@ -257,7 +272,8 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig mergeThreadPriority, reportParseExceptions, handoffConditionTimeout, - alertTimeout + alertTimeout, + segmentWriteOutMediumFactory ); } @@ -278,7 +294,8 @@ public class RealtimeTuningConfig implements TuningConfig, AppenderatorConfig mergeThreadPriority, reportParseExceptions, handoffConditionTimeout, - alertTimeout + alertTimeout, + segmentWriteOutMediumFactory ); } } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java index 4b2c99f644c..286470cae76 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorConfig.java @@ -19,9 +19,11 @@ package io.druid.segment.realtime.appenderator; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.File; public interface AppenderatorConfig @@ -37,4 +39,7 @@ public interface AppenderatorConfig IndexSpec getIndexSpec(); File getBasePersistDirectory(); + + @Nullable + SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory(); } diff --git a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java index 537cf5ae28a..aec0f751a3b 100644 --- a/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/io/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -620,7 +620,8 @@ public class AppenderatorImpl implements Appenderator schema.getGranularitySpec().isRollup(), schema.getAggregators(), mergedTarget, - tuningConfig.getIndexSpec() + tuningConfig.getIndexSpec(), + tuningConfig.getSegmentWriteOutMediumFactory() ); } catch (Throwable t) { @@ -1151,7 +1152,8 @@ public class AppenderatorImpl implements Appenderator indexToPersist.getIndex(), identifier.getInterval(), new File(persistDir, String.valueOf(indexToPersist.getCount())), - indexSpec + indexSpec, + tuningConfig.getSegmentWriteOutMediumFactory() ); indexToPersist.swapSegment( diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java index 224d3c7395d..546181b1fb0 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/FlushingPlumber.java @@ -29,8 +29,8 @@ import io.druid.common.guava.ThreadRenamingCallable; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.concurrent.ScheduledExecutors; +import io.druid.java.util.common.granularity.Granularity; import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.segment.IndexIO; import io.druid.segment.IndexMerger; diff --git a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java index d4477235431..3a19ed63216 100644 --- a/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/io/druid/segment/realtime/plumber/RealtimePlumber.java @@ -422,13 +422,13 @@ public class RealtimePlumber implements Plumber closer.register(segmentAndCloseable.rhs); } - mergedFile = indexMerger.mergeQueryableIndex( indexes, schema.getGranularitySpec().isRollup(), schema.getAggregators(), mergedTarget, - config.getIndexSpec() + config.getIndexSpec(), + config.getSegmentWriteOutMediumFactory() ); } catch (Throwable t) { @@ -942,7 +942,8 @@ public class RealtimePlumber implements Plumber indexToPersist.getIndex(), interval, new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())), - indexSpec + indexSpec, + config.getSegmentWriteOutMediumFactory() ); indexToPersist.swapSegment( diff --git a/server/src/main/java/io/druid/server/security/Escalator.java b/server/src/main/java/io/druid/server/security/Escalator.java index 021572ed3f0..c3a9c19aa7d 100644 --- a/server/src/main/java/io/druid/server/security/Escalator.java +++ b/server/src/main/java/io/druid/server/security/Escalator.java @@ -23,14 +23,14 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.metamx.http.client.HttpClient; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopEscalator.class) -@JsonSubTypes(value = { - @JsonSubTypes.Type(name = "noop", value = NoopEscalator.class), -}) /** * This interface provides methods needed for escalating internal system requests with priveleged authentication * credentials. Each Escalator is associated with a specific authentication scheme, like Authenticators. */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopEscalator.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "noop", value = NoopEscalator.class), +}) public interface Escalator { /** diff --git a/server/src/test/java/io/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java b/server/src/test/java/io/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java index ed10455f337..ffacdb5f6a9 100644 --- a/server/src/test/java/io/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java +++ b/server/src/test/java/io/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java @@ -28,6 +28,9 @@ import com.google.common.collect.Lists; import com.metamx.emitter.EmittingLogger; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.segment.TestHelper; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; @@ -37,21 +40,36 @@ import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; +import java.io.IOException; +import java.util.Collection; import java.util.List; +@RunWith(Parameterized.class) public class SegmentLoaderLocalCacheManagerTest { + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} + ); + } + @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); private final ObjectMapper jsonMapper; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private File localSegmentCacheFolder; private SegmentLoaderLocalCacheManager manager; - public SegmentLoaderLocalCacheManagerTest() + public SegmentLoaderLocalCacheManagerTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { jsonMapper = new DefaultObjectMapper(); jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); @@ -61,6 +79,7 @@ public class SegmentLoaderLocalCacheManagerTest new LocalDataSegmentPuller() ) ); + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Before @@ -76,7 +95,7 @@ public class SegmentLoaderLocalCacheManagerTest locations.add(locationConfig); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -150,7 +169,7 @@ public class SegmentLoaderLocalCacheManagerTest locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -203,7 +222,7 @@ public class SegmentLoaderLocalCacheManagerTest locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -258,7 +277,7 @@ public class SegmentLoaderLocalCacheManagerTest locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -312,7 +331,7 @@ public class SegmentLoaderLocalCacheManagerTest locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); diff --git a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java index 137b75b451f..ec023fb108b 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -29,6 +29,7 @@ import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.TestHelper; @@ -110,8 +111,8 @@ public class FireDepartmentTest null, null, null, - TestHelper.getTestIndexMergerV9(), - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()), + TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()), MapCache.create(0), NO_CACHE_CONFIG, TestHelper.getJsonMapper() diff --git a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java index 2e483fb95b3..59f145555e8 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -213,6 +213,7 @@ public class RealtimeManagerTest 0, null, null, + null, null ); plumber = new TestPlumber(new Sink( @@ -271,6 +272,7 @@ public class RealtimeManagerTest 0, null, null, + null, null ); @@ -289,6 +291,7 @@ public class RealtimeManagerTest 0, null, null, + null, null ); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java index 3a6dc6dcfb0..d7b0aa4a647 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorPlumberTest.java @@ -81,6 +81,7 @@ public class AppenderatorPlumberTest 0, false, null, + null, null ); diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 766fa19709b..555e05d2067 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java @@ -33,6 +33,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; @@ -146,6 +147,7 @@ public class AppenderatorTester implements AutoCloseable 0, null, null, + null, null ); @@ -154,6 +156,7 @@ public class AppenderatorTester implements AutoCloseable indexIO = new IndexIO( objectMapper, + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -163,7 +166,7 @@ public class AppenderatorTester implements AutoCloseable } } ); - indexMerger = new IndexMergerV9(objectMapper, indexIO); + indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); emitter = new ServiceEmitter( "test", diff --git a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java index 39bc73eaa57..08157eaf7f6 100644 --- a/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java +++ b/server/src/test/java/io/druid/segment/realtime/appenderator/DefaultOfflineAppenderatorFactoryTest.java @@ -147,6 +147,7 @@ public class DefaultOfflineAppenderatorFactoryTest 0, null, null, + null, null ); diff --git a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java index 51bd505ea93..e9d9a7579b4 100644 --- a/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java +++ b/server/src/test/java/io/druid/segment/realtime/firehose/IngestSegmentFirehoseTest.java @@ -33,6 +33,9 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.hll.HyperLogLogCollector; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -52,12 +55,17 @@ import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.io.File; +import java.io.IOException; +import java.util.Collection; import java.util.List; /** */ +@RunWith(Parameterized.class) public class IngestSegmentFirehoseTest { private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( @@ -88,11 +96,26 @@ public class IngestSegmentFirehoseTest new HyperUniquesAggregatorFactory("unique_hosts", "unique_hosts") ); + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} + ); + } + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); - private IndexIO indexIO = TestHelper.getTestIndexIO(); - private IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(); + private final IndexIO indexIO; + private final IndexMerger indexMerger; + + public IngestSegmentFirehoseTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) + { + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + } @Test public void testReadFromIndexAndWriteAnotherIndex() throws Exception @@ -207,7 +230,7 @@ public class IngestSegmentFirehoseTest for (String line : rows) { index.add(parser.parse(line)); } - indexMerger.persist(index, segmentDir, new IndexSpec()); + indexMerger.persist(index, segmentDir, new IndexSpec(), null); } } } diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java index f9ddac4413f..efe1dbdef57 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/RealtimePlumberSchoolTest.java @@ -39,6 +39,9 @@ import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; @@ -84,7 +87,24 @@ import java.util.concurrent.TimeUnit; @RunWith(Parameterized.class) public class RealtimePlumberSchoolTest { + @Parameterized.Parameters(name = "rejectionPolicy = {0}, segmentWriteOutMediumFactory = {1}") + public static Collection constructorFeeder() throws IOException + { + final RejectionPolicyFactory[] rejectionPolicies = new RejectionPolicyFactory[]{ + new NoopRejectionPolicyFactory(), + new MessageTimeRejectionPolicyFactory() + }; + + final List constructors = Lists.newArrayList(); + for (RejectionPolicyFactory rejectionPolicy : rejectionPolicies) { + constructors.add(new Object[]{rejectionPolicy, OffHeapMemorySegmentWriteOutMediumFactory.instance()}); + constructors.add(new Object[]{rejectionPolicy, TmpFileSegmentWriteOutMediumFactory.instance()}); + } + return constructors; + } + private final RejectionPolicyFactory rejectionPolicy; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private RealtimePlumber plumber; private RealtimePlumberSchool realtimePlumberSchool; private DataSegmentAnnouncer announcer; @@ -99,24 +119,10 @@ public class RealtimePlumberSchoolTest private FireDepartmentMetrics metrics; private File tmpDir; - public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy) + public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { this.rejectionPolicy = rejectionPolicy; - } - - @Parameterized.Parameters(name = "rejectionPolicy = {0}") - public static Collection constructorFeeder() throws IOException - { - final RejectionPolicyFactory[] rejectionPolicies = new RejectionPolicyFactory[]{ - new NoopRejectionPolicyFactory(), - new MessageTimeRejectionPolicyFactory() - }; - - final List constructors = Lists.newArrayList(); - for (RejectionPolicyFactory rejectionPolicy : rejectionPolicies) { - constructors.add(new Object[]{rejectionPolicy}); - } - return constructors; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Before @@ -204,6 +210,7 @@ public class RealtimePlumberSchoolTest 0, false, null, + null, null ); @@ -215,8 +222,8 @@ public class RealtimePlumberSchoolTest segmentPublisher, handoffNotifierFactory, MoreExecutors.sameThreadExecutor(), - TestHelper.getTestIndexMergerV9(), - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), MapCache.create(0), FireDepartmentTest.NO_CACHE_CONFIG, TestHelper.getJsonMapper() diff --git a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java index 00fcf20d05d..dce3e7a8978 100644 --- a/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java +++ b/server/src/test/java/io/druid/segment/realtime/plumber/SinkTest.java @@ -74,6 +74,7 @@ public class SinkTest 0, null, null, + null, null ); final Sink sink = new Sink( diff --git a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java index 06ae9b7c5d0..49518d6483d 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -182,6 +182,7 @@ public class DruidJsonValidatorTest 0, true, null, + null, null ) ), diff --git a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java index dfdef39e836..96e5dba2870 100644 --- a/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/io/druid/sql/calcite/schema/DruidSchemaTest.java @@ -24,13 +24,13 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import io.druid.data.input.InputRow; import io.druid.java.util.common.Intervals; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import io.druid.segment.IndexBuilder; import io.druid.segment.QueryableIndex; -import io.druid.segment.TestHelper; import io.druid.segment.incremental.IncrementalIndexSchema; import io.druid.server.security.NoopEscalator; import io.druid.sql.calcite.planner.Calcites; @@ -88,7 +88,7 @@ public class DruidSchemaTest final File tmpDir = temporaryFolder.newFolder(); final QueryableIndex index1 = IndexBuilder.create() .tmpDir(new File(tmpDir, "1")) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema( new IncrementalIndexSchema.Builder() .withMetrics( @@ -104,7 +104,7 @@ public class DruidSchemaTest final QueryableIndex index2 = IndexBuilder.create() .tmpDir(new File(tmpDir, "2")) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema( new IncrementalIndexSchema.Builder() .withMetrics(new LongSumAggregatorFactory("m1", "m1")) diff --git a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java index 00a6af769f2..a20f24fe2b5 100644 --- a/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/io/druid/sql/calcite/util/CalciteTests.java @@ -44,6 +44,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.guice.ExpressionModule; import io.druid.guice.annotations.Json; import io.druid.math.expr.ExprMacroTable; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.DruidProcessingConfig; @@ -420,26 +421,29 @@ public class CalciteTests public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmpDir) { - final QueryableIndex index1 = IndexBuilder.create() - .tmpDir(new File(tmpDir, "1")) - .indexMerger(TestHelper.getTestIndexMergerV9()) - .schema(INDEX_SCHEMA) - .rows(ROWS1) - .buildMMappedIndex(); + final QueryableIndex index1 = IndexBuilder + .create() + .tmpDir(new File(tmpDir, "1")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(INDEX_SCHEMA) + .rows(ROWS1) + .buildMMappedIndex(); - final QueryableIndex index2 = IndexBuilder.create() - .tmpDir(new File(tmpDir, "2")) - .indexMerger(TestHelper.getTestIndexMergerV9()) - .schema(INDEX_SCHEMA) - .rows(ROWS2) - .buildMMappedIndex(); + final QueryableIndex index2 = IndexBuilder + .create() + .tmpDir(new File(tmpDir, "2")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(INDEX_SCHEMA) + .rows(ROWS2) + .buildMMappedIndex(); - final QueryableIndex forbiddenIndex = IndexBuilder.create() - .tmpDir(new File(tmpDir, "forbidden")) - .indexMerger(TestHelper.getTestIndexMergerV9()) - .schema(INDEX_SCHEMA) - .rows(FORBIDDEN_ROWS) - .buildMMappedIndex(); + final QueryableIndex forbiddenIndex = IndexBuilder + .create() + .tmpDir(new File(tmpDir, "forbidden")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(INDEX_SCHEMA) + .rows(FORBIDDEN_ROWS) + .buildMMappedIndex(); return new SpecificSegmentsQuerySegmentWalker(queryRunnerFactoryConglomerate()).add( DataSegment.builder()