From 506ef5a3091f9f5afe7c8d3a9ec2988b8191edf6 Mon Sep 17 00:00:00 2001 From: leventov Date: Fri, 1 Sep 2017 19:25:31 -0500 Subject: [PATCH 01/14] Replace IOPeon with OutputMedium; Improve compression --- .../CompressedIndexedIntsBenchmark.java | 24 +- .../CompressedVSizeIndexedBenchmark.java | 11 +- .../benchmark/FilterPartitionBenchmark.java | 7 +- .../FilteredAggregatorBenchmark.java | 7 +- .../benchmark/FloatCompressionBenchmark.java | 2 +- ...loatCompressionBenchmarkFileGenerator.java | 48 +- .../benchmark/GenericIndexedBenchmark.java | 7 +- .../GroupByTypeInterfaceBenchmark.java | 7 +- ...LongCompressionBenchmarkFileGenerator.java | 49 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 7 +- .../benchmark/datagen/SegmentGenerator.java | 7 +- .../indexing/IndexMergeBenchmark.java | 10 +- .../indexing/IndexPersistBenchmark.java | 7 +- .../benchmark/query/GroupByBenchmark.java | 7 +- .../benchmark/query/SearchBenchmark.java | 7 +- .../benchmark/query/SelectBenchmark.java | 7 +- .../benchmark/query/TimeseriesBenchmark.java | 7 +- .../druid/benchmark/query/TopNBenchmark.java | 7 +- .../druid/common/utils/SerializerUtils.java | 92 +--- .../io/druid/io/ByteBufferInputStream.java | 47 +- .../src/main/java/io/druid/io/Channels.java | 26 +- .../content/configuration/indexing-service.md | 21 + .../extensions-core/kafka-ingestion.md | 1 + docs/content/ingestion/stream-pull.md | 1 + docs/content/ingestion/tasks.md | 3 +- .../theta/SketchMergeComplexMetricSerde.java | 6 +- .../ApproximateHistogramFoldingSerde.java | 6 +- .../sql/QuantileSqlAggregatorTest.java | 4 +- .../indexing/kafka/KafkaTuningConfig.java | 93 ++-- .../kafka/supervisor/KafkaSupervisorSpec.java | 1 + .../KafkaSupervisorTuningConfig.java | 7 +- .../indexing/kafka/KafkaIndexTaskTest.java | 3 +- .../indexing/kafka/KafkaTuningConfigTest.java | 1 + .../kafka/supervisor/KafkaSupervisorTest.java | 1 + .../aggregation/variance/VarianceSerde.java | 6 +- .../io/druid/indexer/IndexGeneratorJob.java | 4 +- .../indexer/updater/HadoopConverterJob.java | 3 +- .../updater/HadoopConverterJobTest.java | 10 +- .../common/index/YeOldePlumberSchool.java | 12 +- .../indexing/common/task/AppendTask.java | 7 +- ...ConvertSegmentBackwardsCompatibleTask.java | 14 +- .../common/task/ConvertSegmentTask.java | 142 ++++-- .../common/task/HadoopConverterTask.java | 16 +- .../druid/indexing/common/task/IndexTask.java | 29 +- .../druid/indexing/common/task/MergeTask.java | 7 +- .../indexing/common/task/MergeTaskBase.java | 18 +- .../common/task/SameIntervalMergeTask.java | 11 +- .../io/druid/indexing/common/TestUtils.java | 4 +- .../common/task/ConvertSegmentTaskTest.java | 4 +- .../task/HadoopConverterTaskSerDeTest.java | 16 +- .../indexing/common/task/IndexTaskTest.java | 1 + .../common/task/MergeTaskBaseTest.java | 2 +- .../common/task/RealtimeIndexTaskTest.java | 1 + .../task/SameIntervalMergeTaskTest.java | 1 + .../indexing/common/task/TaskSerdeTest.java | 37 +- .../IngestSegmentFirehoseFactoryTest.java | 2 +- ...estSegmentFirehoseFactoryTimelineTest.java | 2 +- .../indexing/overlord/TaskLifecycleTest.java | 7 +- .../util/common/io/smoosh/FileSmoosher.java | 19 +- pom.xml | 4 +- processing/pom.xml | 9 + .../druid/output/ByteBufferOutputBytes.java | 228 ++++++++++ .../output/DirectByteBufferOutputBytes.java | 52 +++ .../java/io/druid/output/FileOutputBytes.java | 135 ++++++ .../output/HeapByteBufferOutputBytes.java | 31 ++ .../output/OffHeapMemoryOutputMedium.java | 49 ++ .../OffHeapMemoryOutputMediumFactory.java | 44 ++ .../java/io/druid/output/OutputBytes.java | 68 +++ .../java/io/druid/output/OutputMedium.java | 45 ++ .../io/druid/output/OutputMediumFactory.java | 48 ++ .../io/druid/output/OutputMediumModule.java | 33 ++ .../io/druid/output/TmpFileOutputMedium.java | 68 +++ .../output/TmpFileOutputMediumFactory.java | 43 ++ .../hyperloglog/HyperUniquesSerde.java | 6 +- .../CompressedVSizeIndexedSupplier.java | 46 +- .../CompressedVSizeIndexedV3Supplier.java | 45 +- .../io/druid/segment/DimensionHandler.java | 8 +- .../druid/segment/DoubleColumnSerializer.java | 37 +- .../druid/segment/DoubleDimensionHandler.java | 11 +- .../segment/DoubleDimensionMergerV9.java | 49 +- .../io/druid/segment/FloatColumnSelector.java | 5 - .../druid/segment/FloatColumnSerializer.java | 37 +- .../druid/segment/FloatDimensionHandler.java | 13 +- .../druid/segment/FloatDimensionMergerV9.java | 21 +- .../segment/GenericColumnSerializer.java | 10 +- .../main/java/io/druid/segment/IndexIO.java | 16 +- .../java/io/druid/segment/IndexMerger.java | 52 ++- .../java/io/druid/segment/IndexMergerV9.java | 196 ++++---- .../main/java/io/druid/segment/IndexSpec.java | 30 +- .../druid/segment/LongColumnSerializer.java | 37 +- .../druid/segment/LongDimensionHandler.java | 11 +- .../druid/segment/LongDimensionMergerV9.java | 21 +- .../java/io/druid/segment/MetricHolder.java | 120 +---- .../main/java/io/druid/segment/Rowboat.java | 2 +- .../druid/segment/StringDimensionHandler.java | 8 +- .../segment/StringDimensionMergerV9.java | 215 ++++----- .../segment/column/ColumnDescriptor.java | 19 +- .../BlockLayoutDoubleSupplierSerializer.java | 107 ++--- .../BlockLayoutFloatSupplierSerializer.java | 108 ++--- .../BlockLayoutIndexedDoubleSupplier.java | 11 +- .../data/BlockLayoutIndexedFloatSupplier.java | 12 +- .../data/BlockLayoutIndexedLongSupplier.java | 14 +- .../BlockLayoutLongSupplierSerializer.java | 118 ++--- .../segment/data/ByteBufferSerializer.java | 8 +- .../druid/segment/data/ByteBufferWriter.java | 94 +--- .../CompressedByteBufferObjectStrategy.java | 70 --- .../CompressedDoubleBufferObjectStrategy.java | 73 --- .../CompressedDoublesIndexedSupplier.java | 18 +- .../CompressedFloatBufferObjectStrategy.java | 72 --- .../data/CompressedFloatsIndexedSupplier.java | 51 ++- .../CompressedIntBufferObjectStrategy.java | 71 --- .../data/CompressedIntsIndexedSupplier.java | 172 +++---- .../data/CompressedIntsIndexedWriter.java | 107 +++-- .../CompressedLongBufferObjectStrategy.java | 72 --- .../data/CompressedLongsIndexedSupplier.java | 48 +- .../data/CompressedObjectStrategy.java | 395 ---------------- .../data/CompressedVSizeIndexedV3Writer.java | 69 ++- .../CompressedVSizeIntsIndexedSupplier.java | 130 +++--- .../CompressedVSizeIntsIndexedWriter.java | 144 +++--- .../segment/data/CompressionFactory.java | 81 ++-- .../segment/data/CompressionStrategy.java | 366 +++++++++++++++ .../data/ConciseBitmapSerdeFactory.java | 8 +- ...DecompressingByteBufferObjectStrategy.java | 84 ++++ .../segment/data/DeltaLongEncodingWriter.java | 22 +- .../data/DoubleSupplierSerializer.java | 11 +- .../EntireLayoutDoubleSupplierSerializer.java | 72 +-- .../EntireLayoutFloatSupplierSerializer.java | 86 ++-- .../EntireLayoutLongSupplierSerializer.java | 75 +--- .../segment/data/FloatSupplierSerializer.java | 10 +- .../io/druid/segment/data/GenericIndexed.java | 98 ++-- .../segment/data/GenericIndexedWriter.java | 424 ++++++++---------- .../druid/segment/data/IndexedIntsWriter.java | 10 +- .../io/druid/segment/data/IndexedRTree.java | 3 +- .../IntermediateLongSupplierSerializer.java | 75 +--- .../segment/data/LongSupplierSerializer.java | 10 +- .../segment/data/LongsLongEncodingWriter.java | 13 +- .../data/MultiValueIndexedIntsWriter.java | 12 +- .../io/druid/segment/data/ObjectStrategy.java | 13 +- .../data/RoaringBitmapSerdeFactory.java | 3 +- .../segment/data/TableLongEncodingWriter.java | 24 +- .../io/druid/segment/data/TmpFileIOPeon.java | 100 ----- .../data/VSizeCompressedObjectStrategy.java | 88 ---- .../io/druid/segment/data/VSizeIndexed.java | 95 ++-- .../druid/segment/data/VSizeIndexedInts.java | 124 +++-- .../segment/data/VSizeIndexedIntsWriter.java | 66 +-- .../segment/data/VSizeIndexedWriter.java | 186 ++++---- .../io/druid/segment/data/VSizeLongSerde.java | 20 + .../druid/segment/data/WritableSupplier.java | 8 +- .../druid/segment/serde/ColumnPartSerde.java | 10 - .../segment/serde/ComplexColumnPartSerde.java | 20 +- .../serde/ComplexColumnSerializer.java | 53 +-- .../segment/serde/ComplexMetricSerde.java | 8 +- .../DictionaryEncodedColumnPartSerde.java | 21 +- .../serde/DoubleGenericColumnPartSerde.java | 91 ++-- .../serde/FloatGenericColumnPartSerde.java | 28 +- .../serde/FloatGenericColumnSupplier.java | 6 +- ...olumnSupportedComplexColumnSerializer.java | 65 ++- .../serde/LongGenericColumnPartSerde.java | 23 +- .../io/druid/segment/serde/Serializer.java | 31 ++ .../druid/query/MultiValuedDimensionTest.java | 57 +-- .../aggregation/AggregationTestHelper.java | 21 +- .../aggregation/FilteredAggregatorTest.java | 6 +- .../groupby/GroupByMultiSegmentTest.java | 10 +- .../java/io/druid/segment/AppendTest.java | 8 +- .../CompressedVSizeIndexedV3SupplierTest.java | 15 +- .../ConciseBitmapIndexMergerV9Test.java | 16 +- .../java/io/druid/segment/EmptyIndexTest.java | 30 +- .../java/io/druid/segment/IndexBuilder.java | 24 +- .../java/io/druid/segment/IndexIOTest.java | 9 +- .../io/druid/segment/IndexMergerTestBase.java | 347 ++++++++------ .../IndexMergerV9CompatibilityTest.java | 50 ++- .../IndexMergerV9WithSpatialIndexTest.java | 67 +-- .../java/io/druid/segment/IndexSpecTest.java | 12 +- .../QueryableIndexIndexableAdapterTest.java | 45 +- .../RoaringBitmapIndexMergerV9Test.java | 16 +- .../io/druid/segment/SchemalessIndexTest.java | 77 ++-- .../druid/segment/SchemalessTestFullTest.java | 36 +- .../segment/SchemalessTestSimpleTest.java | 34 +- .../segment/StringDimensionHandlerTest.java | 6 +- .../java/io/druid/segment/TestHelper.java | 29 +- .../test/java/io/druid/segment/TestIndex.java | 15 +- .../data/CompressedFloatsSerdeTest.java | 32 +- .../CompressedIntsIndexedSupplierTest.java | 15 +- .../data/CompressedIntsIndexedWriterTest.java | 115 +++-- .../data/CompressedLongsSerdeTest.java | 30 +- .../CompressedVSizeIndexedSupplierTest.java | 15 +- .../CompressedVSizeIndexedV3WriterTest.java | 105 ++--- ...ompressedVSizeIntsIndexedSupplierTest.java | 26 +- .../CompressedVSizeIntsIndexedWriterTest.java | 65 +-- .../segment/data/CompressionStrategyTest.java | 54 ++- .../segment/data/GenericIndexedTest.java | 2 +- .../druid/segment/data/IOPeonForTesting.java | 76 ---- .../segment/data/VSizeIndexedIntsTest.java | 16 +- .../data/VSizeIndexedIntsWriterTest.java | 24 +- .../druid/segment/data/VSizeIndexedTest.java | 2 +- .../druid/segment/filter/BaseFilterTest.java | 21 +- .../filter/SpatialFilterBonusTest.java | 66 +-- .../segment/filter/SpatialFilterTest.java | 16 +- .../IncrementalIndexAdapterTest.java | 6 +- .../loading/SegmentizerFactoryTest.java | 21 +- .../serde/HyperUniquesSerdeForTest.java | 6 +- ...nSupportedComplexColumnSerializerTest.java | 11 +- .../druid/initialization/Initialization.java | 2 + .../indexing/RealtimeTuningConfig.java | 25 +- .../appenderator/AppenderatorConfig.java | 5 + .../appenderator/AppenderatorImpl.java | 6 +- .../realtime/plumber/FlushingPlumber.java | 2 +- .../realtime/plumber/RealtimePlumber.java | 6 +- .../SegmentLoaderLocalCacheManagerTest.java | 31 +- .../segment/realtime/FireDepartmentTest.java | 5 +- .../segment/realtime/RealtimeManagerTest.java | 3 + .../appenderator/AppenderatorPlumberTest.java | 1 + .../appenderator/AppenderatorTester.java | 5 +- ...DefaultOfflineAppenderatorFactoryTest.java | 1 + .../firehose/IngestSegmentFirehoseTest.java | 29 +- .../plumber/RealtimePlumberSchoolTest.java | 43 +- .../segment/realtime/plumber/SinkTest.java | 1 + .../cli/validate/DruidJsonValidatorTest.java | 1 + .../sql/calcite/schema/DruidSchemaTest.java | 6 +- .../druid/sql/calcite/util/CalciteTests.java | 5 +- 220 files changed, 4662 insertions(+), 4464 deletions(-) rename processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java => common/src/main/java/io/druid/io/ByteBufferInputStream.java (53%) rename processing/src/main/java/io/druid/segment/data/IOPeon.java => common/src/main/java/io/druid/io/Channels.java (69%) create mode 100644 processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java create mode 100644 processing/src/main/java/io/druid/output/DirectByteBufferOutputBytes.java create mode 100644 processing/src/main/java/io/druid/output/FileOutputBytes.java create mode 100644 processing/src/main/java/io/druid/output/HeapByteBufferOutputBytes.java create mode 100644 processing/src/main/java/io/druid/output/OffHeapMemoryOutputMedium.java create mode 100644 processing/src/main/java/io/druid/output/OffHeapMemoryOutputMediumFactory.java create mode 100644 processing/src/main/java/io/druid/output/OutputBytes.java create mode 100644 processing/src/main/java/io/druid/output/OutputMedium.java create mode 100644 processing/src/main/java/io/druid/output/OutputMediumFactory.java create mode 100644 processing/src/main/java/io/druid/output/OutputMediumModule.java create mode 100644 processing/src/main/java/io/druid/output/TmpFileOutputMedium.java create mode 100644 processing/src/main/java/io/druid/output/TmpFileOutputMediumFactory.java delete mode 100644 processing/src/main/java/io/druid/segment/data/CompressedByteBufferObjectStrategy.java delete mode 100644 processing/src/main/java/io/druid/segment/data/CompressedDoubleBufferObjectStrategy.java delete mode 100644 processing/src/main/java/io/druid/segment/data/CompressedFloatBufferObjectStrategy.java delete mode 100644 processing/src/main/java/io/druid/segment/data/CompressedIntBufferObjectStrategy.java delete mode 100644 processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java delete mode 100644 processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java create mode 100644 processing/src/main/java/io/druid/segment/data/CompressionStrategy.java create mode 100644 processing/src/main/java/io/druid/segment/data/DecompressingByteBufferObjectStrategy.java delete mode 100644 processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java delete mode 100644 processing/src/main/java/io/druid/segment/data/VSizeCompressedObjectStrategy.java create mode 100644 processing/src/main/java/io/druid/segment/serde/Serializer.java delete mode 100644 processing/src/test/java/io/druid/segment/data/IOPeonForTesting.java diff --git a/benchmarks/src/main/java/io/druid/benchmark/CompressedIndexedIntsBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/CompressedIndexedIntsBenchmark.java index 307ef5d67d4a..23c4adb57ddb 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 void setup() throws IOException } 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 void close() throws IOException } }; - 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 c0b1c7f10973..da42f154e8da 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 @@ 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,7 +96,9 @@ public IndexedInts apply(int[] input) } ), bound - 1, - ByteOrder.nativeOrder(), CompressedObjectStrategy.CompressionStrategy.LZ4 + ByteOrder.nativeOrder(), + CompressionStrategy.LZ4, + Closer.create() ) ); this.compressed = CompressedVSizeIndexedSupplier.fromByteBuffer( @@ -117,7 +120,7 @@ public VSizeIndexedInts apply(int[] input) } } ) - ).asWritableSupplier() + ) ); this.uncompressed = VSizeIndexed.readFromByteBuffer(bufferUncompressed); @@ -159,7 +162,7 @@ public void close() throws IOException } }; - 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 0ec205463d39..ce8130626661 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.js.JavaScriptConfig; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -141,7 +143,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } @Setup @@ -178,7 +180,8 @@ public void setup() throws IOException 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 87fa11c3ba32..10d2e8fc9925 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.js.JavaScriptConfig; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -137,6 +138,7 @@ public class FilteredAggregatorBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -146,7 +148,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } @Setup @@ -203,7 +205,8 @@ public void setup() throws IOException 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 b06ecb0657a2..ee46f8384f79 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 void setup() throws Exception 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 f66a14f420ba..41e6edb97a9c 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.output.OffHeapMemoryOutputMedium; 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 static void main(String[] args) throws IOException, URISyntaxException // 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 OffHeapMemoryOutputMedium(), "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 4b5fb28574d5..cd9f10396cac 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.primitives.Ints; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.output.OffHeapMemoryOutputMedium; 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 int compare(byte[] o1, byte[] o2) public void createGenericIndexed() throws IOException { GenericIndexedWriter genericIndexedWriter = new GenericIndexedWriter<>( - new TmpFileIOPeon(), + new OffHeapMemoryOutputMedium(), "genericIndexedBenchmark", byteArrayStrategy ); @@ -121,14 +121,13 @@ public void createGenericIndexed() throws IOException 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 aada2596eacc..08377d065595 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -162,7 +164,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -339,7 +341,8 @@ public void setup() throws IOException 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/LongCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java index bd00dfa62c00..a4a1873f1612 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.output.OffHeapMemoryOutputMedium; 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 static void main(String[] args) throws IOException, URISyntaxException // 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 static void main(String[] args) throws IOException, URISyntaxException compFile.delete(); File dataFile = new File(dir, entry.getKey()); - TmpFileIOPeon iopeon = new TmpFileIOPeon(true); LongSupplierSerializer writer = CompressionFactory.getLongSerializer( - iopeon, + new OffHeapMemoryOutputMedium(), "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 b62ac5e28eb8..2ddc7bfad733 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -145,7 +147,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -291,7 +293,8 @@ public void setup() throws IOException 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 78ba8ff70818..67a986301380 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java @@ -32,6 +32,7 @@ import io.druid.hll.HyperLogLogHash; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexBuilder; @@ -141,8 +142,8 @@ public QueryableIndex generate( return Iterables.getOnlyElement(indexes); } else { try { - final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex( - TestHelper.getTestIndexMergerV9().merge( + final QueryableIndex merged = TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()).loadIndex( + TestHelper.getTestIndexMergerV9(OffHeapMemoryOutputMediumFactory.instance()).merge( indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()), false, schemaInfo.getAggs() @@ -183,7 +184,7 @@ private QueryableIndex makeIndex( .create() .schema(indexSchema) .tmpDir(new File(new File(tempDir, identifier), String.valueOf(indexNumber))) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .outputMediumFactory(OffHeapMemoryOutputMediumFactory.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 842c7245e3df..abe004bbd554 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -28,6 +28,7 @@ import io.druid.hll.HyperLogLogHash; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; @@ -90,6 +91,7 @@ public class IndexMergeBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -99,7 +101,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } @Setup @@ -139,7 +141,8 @@ public void setup() throws IOException File indexFile = INDEX_MERGER_V9.persist( incIndex, tmpDir, - new IndexSpec() + new IndexSpec(), + null ); QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); @@ -183,7 +186,8 @@ public void mergeV9(Blackhole blackhole) throws Exception 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 c5e41f08fd4c..108696bdd1f7 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.hll.HyperLogLogHash; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -97,7 +99,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } @Setup @@ -172,7 +174,8 @@ public void persistV9(Blackhole blackhole) throws Exception 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 20f0e46c12bb..d99bd38b76f9 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -167,7 +169,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -434,7 +436,8 @@ public void setup() throws IOException 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 babe91f61d7e..816e81265c7b 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.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -149,7 +151,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -360,7 +362,8 @@ public void setup() throws IOException 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 39da308ee633..725de6ba64c5 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.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -143,7 +145,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -223,7 +225,8 @@ public void setup() throws IOException 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 d428352738ea..cd476213e908 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.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -141,7 +143,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -286,7 +288,8 @@ public void setup() throws IOException 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 d29c90f5717f..bc3799d66212 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -142,7 +144,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); @@ -263,7 +265,8 @@ public void setup() throws IOException 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/SerializerUtils.java b/common/src/main/java/io/druid/common/utils/SerializerUtils.java index 9191391f44c4..0ead88af0769 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.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 @@ 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 static void writeBigEndianIntToOutputStream(OutputStream out, int value, 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 void writeString(WritableByteChannel out, String name) throws IOException { 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 byte[] readBytes(ByteBuffer in, int length) throws IOException 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 void writeStrings(OutputStream out, List names) throws IOExceptio } } - public String[] readStrings(InputStream in) throws IOException + String[] readStrings(InputStream in) throws IOException { int length = readInt(in); @@ -171,7 +125,7 @@ public String[] readStrings(InputStream in) throws IOException return retVal; } - public String[] readStrings(ByteBuffer in) throws IOException + String[] readStrings(ByteBuffer in) throws IOException { int length = in.getInt(); @@ -184,20 +138,20 @@ public String[] readStrings(ByteBuffer in) throws IOException 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 int readInt(InputStream in) throws IOException 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 void writeInts(OutputStream out, int[] ints) throws IOException } } - public int[] readInts(InputStream in) throws IOException + int[] readInts(InputStream in) throws IOException { int size = readInt(in); @@ -227,7 +181,7 @@ public int[] readInts(InputStream in) throws IOException 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 void writeLong(WritableByteChannel out, long longValue) throws IOExceptio 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 long readLong(InputStream in) throws IOException 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 void writeLongs(OutputStream out, long[] longs) throws IOException } } - public long[] readLongs(InputStream in) throws IOException + long[] readLongs(InputStream in) throws IOException { int size = readInt(in); @@ -275,20 +229,20 @@ public void writeFloat(OutputStream out, float floatValue) throws IOException 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 void writeFloats(OutputStream out, float[] floats) throws IOException } } - public float[] readFloats(InputStream in) throws IOException + float[] readFloats(InputStream in) throws IOException { int size = readInt(in); diff --git a/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java b/common/src/main/java/io/druid/io/ByteBufferInputStream.java similarity index 53% rename from processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java rename to common/src/main/java/io/druid/io/ByteBufferInputStream.java index afdd1fc99a38..3d9f732f47b6 100644 --- a/processing/src/main/java/io/druid/segment/data/FixedSizeCompressedObjectStrategy.java +++ b/common/src/main/java/io/druid/io/ByteBufferInputStream.java @@ -17,41 +17,48 @@ * under the License. */ -package io.druid.segment.data; +package io.druid.io; -import java.nio.Buffer; +import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; -import java.nio.ByteOrder; -public abstract class FixedSizeCompressedObjectStrategy extends CompressedObjectStrategy +public final class ByteBufferInputStream extends InputStream { - private final int sizePer; - - protected FixedSizeCompressedObjectStrategy( - ByteOrder order, - BufferConverter converter, - CompressionStrategy compression, - int sizePer - ) + private final ByteBuffer buffer; + + public ByteBufferInputStream(ByteBuffer buffer) { - super(order, converter, compression); - this.sizePer = sizePer; + this.buffer = buffer; } - public int getSize() + @Override + public int read() { - return sizePer; + if (!buffer.hasRemaining()) { + return -1; + } + return buffer.get() & 0xFF; } @Override - protected ByteBuffer bufferFor(T val) + public int read(byte[] bytes, int off, int len) { - return ByteBuffer.allocate(converter.sizeOf(getSize())).order(order); + if (len == 0) { + return 0; + } + if (!buffer.hasRemaining()) { + return -1; + } + + len = Math.min(len, buffer.remaining()); + buffer.get(bytes, off, len); + return len; } @Override - protected void decompress(ByteBuffer buffer, int numBytes, ByteBuffer buf) + public int available() throws IOException { - decompressor.decompress(buffer, numBytes, buf, converter.sizeOf(getSize())); + 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 69% rename from processing/src/main/java/io/druid/segment/data/IOPeon.java rename to common/src/main/java/io/druid/io/Channels.java index 38c2eff0b658..141da64d0fc8 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 { - public OutputStream makeOutputStream(String filename) throws IOException; - public InputStream makeInputStream(String filename) throws IOException; - public 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 1631378d7cbb..2e0383d4996b 100644 --- a/docs/content/configuration/indexing-service.md +++ b/docs/content/configuration/indexing-service.md @@ -343,3 +343,24 @@ 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| + +##### OutputMediumFactory + +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. + +For most types of tasks OutputMediumFactory 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.defaultOutputMediumFactory`|`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 7212d18b55cb..c69fe6d9af84 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)| +|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.defaultOutputMediumFactory` is used)| #### IndexSpec diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index 7cd73c441af6..b4d2d5cd4c54 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -155,6 +155,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)| +|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.defaultOutputMediumFactory` 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 318704d34066..8bd209596226 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -120,9 +120,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| +|outputMediumFactory|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|Not specified, the value from `druid.defaultOutputMediumFactory` 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 e0c1ba03bab8..8a2cd7c43246 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 @@ import com.yahoo.sketches.theta.Sketch; import io.druid.data.input.InputRow; +import io.druid.output.OutputMedium; 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 ObjectStrategy getObjectStrategy() } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, 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 b1ad2f883b11..db739657098e 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 @@ import com.google.common.collect.Ordering; import io.druid.data.input.InputRow; +import io.druid.output.OutputMedium; 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 void deserializeColumn( } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, 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 2fed81cabdeb..18f065fa3e9f 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.Iterables; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.guava.Sequences; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.Druids; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -40,7 +41,6 @@ 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; @@ -90,7 +90,7 @@ public void setUp() throws Exception final QueryableIndex index = IndexBuilder.create() .tmpDir(temporaryFolder.newFolder()) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .outputMediumFactory(OffHeapMemoryOutputMediumFactory.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 e9dc4463f80e..e91e48e1b98d 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 @@ -21,13 +21,16 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.output.OutputMediumFactory; 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 org.joda.time.Period; +import javax.annotation.Nullable; import java.io.File; +import java.util.Objects; public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig { @@ -44,6 +47,8 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig @Deprecated private final long handoffConditionTimeout; private final boolean resetOffsetAutomatically; + @Nullable + private final OutputMediumFactory outputMediumFactory; @JsonCreator public KafkaTuningConfig( @@ -57,8 +62,9 @@ public KafkaTuningConfig( @JsonProperty("buildV9Directly") Boolean buildV9Directly, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, - @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically - ) + @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + ) { // Cannot be a static because default basePersistDirectory is unique per-instance final RealtimeTuningConfig defaults = RealtimeTuningConfig.makeDefaultTuningConfig(basePersistDirectory); @@ -80,6 +86,7 @@ public KafkaTuningConfig( this.resetOffsetAutomatically = resetOffsetAutomatically == null ? DEFAULT_RESET_OFFSET_AUTOMATICALLY : resetOffsetAutomatically; + this.outputMediumFactory = outputMediumFactory; } public static KafkaTuningConfig copyOf(KafkaTuningConfig config) @@ -94,7 +101,8 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config) true, config.reportParseExceptions, config.handoffConditionTimeout, - config.resetOffsetAutomatically + config.resetOffsetAutomatically, + config.outputMediumFactory ); } @@ -169,6 +177,14 @@ public boolean isResetOffsetAutomatically() return resetOffsetAutomatically; } + @Override + @JsonProperty + @Nullable + public OutputMediumFactory getOutputMediumFactory() + { + return outputMediumFactory; + } + public KafkaTuningConfig withBasePersistDirectory(File dir) { return new KafkaTuningConfig( @@ -181,7 +197,8 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) true, reportParseExceptions, handoffConditionTimeout, - resetOffsetAutomatically + resetOffsetAutomatically, + outputMediumFactory ); } @@ -197,7 +214,8 @@ public KafkaTuningConfig withMaxRowsInMemory(int rows) true, reportParseExceptions, handoffConditionTimeout, - resetOffsetAutomatically + resetOffsetAutomatically, + outputMediumFactory ); } @@ -210,54 +228,34 @@ public boolean equals(Object o) 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(outputMediumFactory, that.outputMediumFactory); } @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, + outputMediumFactory + ); } @Override @@ -273,6 +271,7 @@ public String toString() ", reportParseExceptions=" + reportParseExceptions + ", handoffConditionTimeout=" + handoffConditionTimeout + ", resetOffsetAutomatically=" + resetOffsetAutomatically + + ", outputMediumFactory=" + outputMediumFactory + '}'; } } 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 c476b05e1053..a8a1b7556a48 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 KafkaSupervisorSpec( 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 ef83165e8ebd..492f6c8df004 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 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexing.kafka.KafkaTuningConfig; +import io.druid.output.OutputMediumFactory; 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 KafkaSupervisorTuningConfig( @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, // for backward compatibility @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, @JsonProperty("workerThreads") Integer workerThreads, @JsonProperty("chatThreads") Integer chatThreads, @JsonProperty("chatRetries") Long chatRetries, @@ -68,7 +71,8 @@ public KafkaSupervisorTuningConfig( // Supervised kafka tasks should respect KafkaSupervisorIOConfig.completionTimeout instead of // handoffConditionTimeout handoffConditionTimeout, - resetOffsetAutomatically + resetOffsetAutomatically, + outputMediumFactory ); this.workerThreads = workerThreads; @@ -128,6 +132,7 @@ public String toString() ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + + ", outputMediumFactory=" + getOutputMediumFactory() + ", 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 ec02dc372cae..e289db5afaf5 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 @@ -1478,7 +1478,8 @@ private KafkaIndexTask createTask( true, reportParseExceptions, handoffConditionTimeout, - resetOffsetAutomatically + resetOffsetAutomatically, + null ); final KafkaIndexTask task = new KafkaIndexTask( taskId, 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 81aa36666836..caa4be08b39b 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 void testCopyOf() throws Exception 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 9ed12b7cbff4..23427371a68d 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 @@ -192,6 +192,7 @@ public void setupTest() throws Exception 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 1ed59fff48c1..1009a7b20012 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 @@ import com.google.common.collect.Ordering; import io.druid.data.input.InputRow; +import io.druid.output.OutputMedium; 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 int compare(VarianceAggregatorCollector o1, VarianceAggregatorCollector o } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, 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 f7194e892eb5..e25aceecafa1 100644 --- a/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/io/druid/indexer/IndexGeneratorJob.java @@ -514,7 +514,7 @@ private File persist( ) throws IOException { return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist( - index, interval, file, config.getIndexSpec(), progressIndicator + index, interval, file, config.getIndexSpec(), progressIndicator, null ); } @@ -527,7 +527,7 @@ protected File mergeQueryableIndex( { 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 d19fc4026fed..eff3c4f4e109 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 @@ protected void map( 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 61350ac9a882..23cb4c760247 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.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; @@ -292,8 +292,8 @@ public MetadataSegmentManagerConfig get() DATASOURCE, interval, new IndexSpec(new RoaringBitmapSerdeFactory(null), - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, - CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.UNCOMPRESSED, + CompressionStrategy.UNCOMPRESSED, CompressionFactory.LongEncodingStrategy.LONGS), oldSemgments, true, @@ -398,8 +398,8 @@ public MetadataSegmentManagerConfig get() 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 808cda53b948..8c20f1292a7d 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 void finishJob() } 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.getOutputMediumFactory() + ); } // Map merged segment so we can extract dimensions @@ -230,7 +237,8 @@ private void spillIfSwappable() indexMergerV9.persist( indexToPersist.getIndex(), dirToPersist, - config.getIndexSpec() + config.getIndexSpec(), + config.getOutputMediumFactory() ); 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 a16a6547d116..253eb6a229be 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 io.druid.indexing.common.TaskToolbox; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.guava.Comparators; +import io.druid.output.OutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; @@ -63,10 +64,11 @@ public AppendTask( @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, @JsonProperty("context") Map context ) { - super(id, dataSource, segments, context); + super(id, dataSource, segments, outputMediumFactory, context); this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.aggregators = aggregators; } @@ -139,7 +141,8 @@ public boolean apply(Rowboat input) adapters, aggregators == null ? null : aggregators.toArray(new AggregatorFactory[aggregators.size()]), outDir, - indexSpec + indexSpec, + getOutputMediumFactory() ); } 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 0a2b9d2a12be..6571aceb0215 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 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.output.OutputMediumFactory; 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,8 +39,9 @@ public ConvertSegmentBackwardsCompatibleTask( @JsonProperty("segment") DataSegment segment, @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, - @JsonProperty("validate") Boolean validate - ) + @JsonProperty("validate") Boolean validate, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + ) { super( id == null ? ConvertSegmentTask.makeId(dataSource, interval) : id, @@ -47,6 +51,7 @@ public ConvertSegmentBackwardsCompatibleTask( indexSpec, force == null ? false : force, validate == null ? false : validate, + outputMediumFactory, null ); } @@ -60,10 +65,11 @@ public SubTask( @JsonProperty("segment") DataSegment segment, @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, - @JsonProperty("validate") Boolean validate + @JsonProperty("validate") Boolean validate, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory ) { - super(groupId, segment, indexSpec, force, validate, null); + super(groupId, segment, indexSpec, force, validate, outputMediumFactory, 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 5ab8251d7f5d..3465d37982f5 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.StringUtils; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OutputMediumFactory; 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 static ConvertSegmentTask create( IndexSpec indexSpec, boolean force, boolean validate, + @Nullable OutputMediumFactory outputMediumFactory, 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, + outputMediumFactory, + context + ); } /** @@ -106,13 +113,24 @@ public static ConvertSegmentTask create( IndexSpec indexSpec, boolean force, boolean validate, + @Nullable OutputMediumFactory outputMediumFactory, 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, + outputMediumFactory, + context + ); } protected static String makeId(String dataSource, Interval interval) @@ -131,22 +149,41 @@ private static ConvertSegmentTask createFromJson( @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory ) { 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, outputMediumFactory, context); } else { - return create(segment, indexSpec, isForce, isValidate, context); + return create(segment, indexSpec, isForce, isValidate, outputMediumFactory, context); } } - return new ConvertSegmentTask(id, dataSource, interval, segment, indexSpec, isForce, isValidate, context); + return new ConvertSegmentTask( + id, + dataSource, + interval, + segment, + indexSpec, + isForce, + isValidate, + outputMediumFactory, + context + ); } - protected ConvertSegmentTask( + @JsonIgnore + private final DataSegment segment; + private final IndexSpec indexSpec; + private final boolean force; + private final boolean validate; + @Nullable + private final OutputMediumFactory outputMediumFactory; + + ConvertSegmentTask( String id, String dataSource, Interval interval, @@ -154,6 +191,7 @@ protected ConvertSegmentTask( IndexSpec indexSpec, boolean force, boolean validate, + @Nullable OutputMediumFactory outputMediumFactory, Map context ) { @@ -162,6 +200,7 @@ protected ConvertSegmentTask( this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.force = force; this.validate = validate; + this.outputMediumFactory = outputMediumFactory; } @JsonProperty @@ -194,6 +233,13 @@ public DataSegment getSegment() return segment; } + @JsonProperty + @Nullable + public OutputMediumFactory getOutputMediumFactory() + { + return outputMediumFactory; + } + @Override public TaskStatus run(TaskToolbox toolbox) throws Exception { @@ -261,7 +307,7 @@ protected Iterable generateSubTasks( @Override public Task apply(DataSegment input) { - return new SubTask(groupId, input, indexSpec, force, validate, context); + return new SubTask(groupId, input, indexSpec, force, validate, outputMediumFactory, context); } } ); @@ -293,6 +339,8 @@ public static class SubTask extends AbstractFixedIntervalTask private final IndexSpec indexSpec; private final boolean force; private final boolean validate; + @Nullable + private final OutputMediumFactory outputMediumFactory; @JsonCreator public SubTask( @@ -301,6 +349,7 @@ public SubTask( @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, @JsonProperty("context") Map context ) { @@ -321,6 +370,7 @@ public SubTask( this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.force = force == null ? false : force; this.validate = validate == null ? true : validate; + this.outputMediumFactory = outputMediumFactory; } @JsonProperty @@ -352,7 +402,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception { 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 TaskStatus run(TaskToolbox toolbox) throws Exception } 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, outputMediumFactory)) { + 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 a83515a6efe1..757f2fdc3115 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.actions.TaskActionClient; import io.druid.java.util.common.UOE; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OutputMediumFactory; 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 HadoopConverterTask( @JsonProperty("jobPriority") String jobPriority, @JsonProperty("segmentOutputPath") String segmentOutputPath, @JsonProperty("classpathPrefix") String classpathPrefix, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, @JsonProperty("context") Map context ) { @@ -81,6 +90,7 @@ public HadoopConverterTask( indexSpec, force, validate == null ? true : validate, + outputMediumFactory, context ); this.hadoopDependencyCoordinates = hadoopDependencyCoordinates; @@ -90,12 +100,6 @@ public HadoopConverterTask( 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 3aa0e5c2444b..bfeffd755355 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 @@ -60,6 +60,7 @@ 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.output.OutputMediumFactory; import io.druid.query.DruidMetrics; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; @@ -921,6 +922,8 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final boolean forceGuaranteedRollup; private final boolean reportParseExceptions; private final long publishTimeout; + @Nullable + private final OutputMediumFactory outputMediumFactory; @JsonCreator public IndexTuningConfig( @@ -936,8 +939,9 @@ public IndexTuningConfig( @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + ) { this( targetPartitionSize, @@ -950,13 +954,14 @@ public IndexTuningConfig( forceGuaranteedRollup, reportParseExceptions, publishTimeout, - null + null, + outputMediumFactory ); } 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( @@ -970,7 +975,8 @@ private IndexTuningConfig( @Nullable Boolean forceGuaranteedRollup, @Nullable Boolean reportParseExceptions, @Nullable Long publishTimeout, - @Nullable File basePersistDirectory + @Nullable File basePersistDirectory, + @Nullable OutputMediumFactory outputMediumFactory ) { Preconditions.checkArgument( @@ -1004,6 +1010,8 @@ private IndexTuningConfig( !(this.forceExtendableShardSpecs && this.forceGuaranteedRollup), "Perfect rollup cannot be guaranteed with extendable shardSpecs" ); + + this.outputMediumFactory = outputMediumFactory; } public IndexTuningConfig withBasePersistDirectory(File dir) @@ -1019,7 +1027,8 @@ public IndexTuningConfig withBasePersistDirectory(File dir) forceGuaranteedRollup, reportParseExceptions, publishTimeout, - dir + dir, + outputMediumFactory ); } @@ -1108,5 +1117,13 @@ public Period getIntermediatePersistPeriod() { return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs } + + @Nullable + @Override + @JsonProperty + public OutputMediumFactory getOutputMediumFactory() + { + return outputMediumFactory; + } } } 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 d897df5c089f..b54f4e02a2de 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.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.indexing.common.TaskToolbox; +import io.druid.output.OutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; @@ -58,10 +59,11 @@ public MergeTask( @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, @JsonProperty("context") Map context ) { - super(id, dataSource, segments, context); + super(id, dataSource, segments, outputMediumFactory, 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 QueryableIndex apply(@Nullable File input) rollup, aggregators.toArray(new AggregatorFactory[aggregators.size()]), outDir, - indexSpec + indexSpec, + getOutputMediumFactory() ); } 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 13c2660aec8c..cef8053109e6 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.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; +import io.druid.output.OutputMediumFactory; import io.druid.segment.IndexIO; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -61,15 +62,19 @@ */ 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 OutputMediumFactory outputMediumFactory; protected MergeTaskBase( final String id, final String dataSource, final List segments, + final @Nullable OutputMediumFactory outputMediumFactory, Map context ) { @@ -104,6 +109,7 @@ public boolean apply(@Nullable DataSegment segment) verifyInputSegments(segments); this.segments = segments; + this.outputMediumFactory = outputMediumFactory; } protected void verifyInputSegments(List segments) @@ -248,6 +254,13 @@ public List getSegments() return segments; } + @JsonProperty + @Nullable + public OutputMediumFactory getOutputMediumFactory() + { + return outputMediumFactory; + } + @Override public String toString() { @@ -256,6 +269,7 @@ public String toString() .add("dataSource", getDataSource()) .add("interval", getInterval()) .add("segments", segments) + .add("outputMediumFactory", outputMediumFactory) .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 b47249eb113c..8a0ac3c4c5cf 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.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUsedAction; import io.druid.java.util.common.DateTimes; +import io.druid.output.OutputMediumFactory; 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 OutputMediumFactory outputMediumFactory; public SameIntervalMergeTask( @JsonProperty("id") String id, @@ -53,6 +57,7 @@ public SameIntervalMergeTask( @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, @JsonProperty("context") Map context ) { @@ -65,6 +70,7 @@ public SameIntervalMergeTask( this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations"); this.rollup = rollup == null ? Boolean.TRUE : rollup; this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; + this.outputMediumFactory = outputMediumFactory; } @JsonProperty("aggregations") @@ -129,6 +135,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception aggregators, rollup, indexSpec, + outputMediumFactory, getContext() ); final TaskStatus status = mergeTask.run(toolbox); @@ -140,13 +147,14 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception public static class SubTask extends MergeTask { - public SubTask( + private SubTask( String baseId, String dataSource, List segments, List aggregators, Boolean rollup, IndexSpec indexSpec, + @Nullable OutputMediumFactory outputMediumFactory, Map context ) { @@ -158,6 +166,7 @@ public SubTask( rollup, indexSpec, true, + outputMediumFactory, 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 4b1070233709..d88b3f71c2c2 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 @@ -29,6 +29,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.math.expr.ExprMacroTable; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.expression.TestExprMacroTable; import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; @@ -54,6 +55,7 @@ public TestUtils() jsonMapper = new DefaultObjectMapper(); indexIO = new IndexIO( jsonMapper, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -63,7 +65,7 @@ public int columnCacheSizeBytes() } } ); - indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO); + indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()); final List list = new ServerModule().getJacksonModules(); for (Module module : list) { 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 a512da941174..be94ec3bcbb6 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 void testSerializationSimple() throws Exception 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 void testSerializationSimple() throws Exception 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 be8885818c02..11a76b70098c 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 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 void testSimpleConverterTaskSerDe() throws IOException PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); final String strOrig = jsonMapper.writeValueAsString(orig); @@ -122,6 +123,7 @@ public void testSimpleSubTaskSerDe() throws IOException PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); HadoopConverterTask.ConverterSubTask subTask = new HadoopConverterTask.ConverterSubTask( @@ -175,6 +177,7 @@ public void testSubTask() PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); HadoopConverterTask.ConverterSubTask subTask = new HadoopConverterTask.ConverterSubTask( @@ -203,6 +206,7 @@ public void testNullValidate() PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); Assert.assertTrue(orig.isValidate()); @@ -223,6 +227,7 @@ public void testMinimal() null, OUTPUT_PATH, null, + null, null ); Assert.assertEquals(DATA_SOURCE, parent.getDataSource()); @@ -248,6 +253,7 @@ public void testGetDataSegment() PRIORITY, OUTPUT_PATH, CLASSPATH_PREFIX, + null, null ); orig.getSegment(); @@ -268,6 +274,7 @@ public void testNull1() null, OUTPUT_PATH, null, + null, null ); } @@ -287,6 +294,7 @@ public void testNull2() null, OUTPUT_PATH, null, + null, null ); } @@ -306,6 +314,7 @@ public void testNull3() null, OUTPUT_PATH, null, + null, null ); } @@ -325,6 +334,7 @@ public void testNull4() 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 0091c40b15d0..9c88db4995e1 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 @@ -1040,6 +1040,7 @@ private static IndexTuningConfig createTuningConfig( 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 0a61e0bdbbd3..a51323803dce 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 c40d8e30ec9c..78949b245ffc 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 @@ -901,6 +901,7 @@ private RealtimeIndexTask makeRealtimeTask(final String taskId, boolean reportPa 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 6e68e8f99526..b3f81277e63a 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 @@ -90,6 +90,7 @@ public void testRun() throws Exception 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 02dfdbbd050a..06d349c30d25 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.IndexTuningConfig; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.output.TmpFileOutputMediumFactory; 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; @@ -188,7 +189,7 @@ public void testIndexTaskSerde() throws Exception 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 ); @@ -250,7 +251,7 @@ public void testIndexTaskwithResourceSerde() throws Exception 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 ); @@ -296,6 +297,7 @@ public void testMergeTaskSerde() throws Exception true, indexSpec, true, + null, null ); @@ -345,6 +347,7 @@ public void testSameIntervalMergeTaskSerde() throws Exception true, indexSpec, true, + null, null ); @@ -412,6 +415,7 @@ public void testVersionConverterTaskSerde() throws Exception null, false, true, + TmpFileOutputMediumFactory.instance(), null ); @@ -427,7 +431,8 @@ public void testVersionConverterTaskSerde() throws Exception 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.getOutputMediumFactory(), task2.getOutputMediumFactory()); } @Test @@ -439,6 +444,7 @@ public void testVersionConverterSubTaskSerde() throws Exception indexSpec, false, true, + null, null ); @@ -500,6 +506,7 @@ public Plumber findPlumber( 0, true, null, + null, null ) ), @@ -564,6 +571,7 @@ public void testAppendTaskSerde() throws Exception ), indexSpec, true, + null, null ); @@ -662,6 +670,7 @@ public void testSegmentConvetSerdeReflection() throws IOException indexSpec, false, true, + TmpFileOutputMediumFactory.instance(), null ); final String json = jsonMapper.writeValueAsString(task); @@ -683,41 +692,43 @@ public void testSegmentConvertSerde() throws IOException 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, + TmpFileOutputMediumFactory.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.getOutputMediumFactory(), convertSegmentTask.getOutputMediumFactory()); } @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 59f216400361..0b50d3d42b32 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 @@ -150,7 +150,7 @@ public static Collection constructorFeeder() throws IOException 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 TaskLockbox tl = new TaskLockbox(ts); 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 25c8fe77e4f8..68d63fffebb3 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 @@ -226,7 +226,7 @@ private static Map persist(File tmpDir, InputRow... rows) } 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 8d38b8abc10c..2e403bef001f 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 @@ -668,7 +668,7 @@ public void testIndexTask() throws Exception 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 ); @@ -725,7 +725,7 @@ public void testIndexTaskFailure() throws Exception 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 ); @@ -1089,7 +1089,7 @@ public void testResumeTasks() throws Exception 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 ); @@ -1210,6 +1210,7 @@ private RealtimeIndexTask newRealtimeIndexTask() 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 4dd66dba8823..2db064d9e6bf 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.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 @@ private SmooshedWriter delegateSmooshedWriter(final String name, final long size 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 e04d669e4219..68389aa3d84d 100644 --- a/pom.xml +++ b/pom.xml @@ -234,7 +234,7 @@ com.ning compress-lzf - 1.0.3 + 1.0.4 io.airlift @@ -699,6 +699,8 @@ asm-commons 5.2 + + org.codehaus.jackson jackson-core-asl diff --git a/processing/pom.xml b/processing/pom.xml index 9479b623c48b..2a83a58cfa1e 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 @@ -97,6 +102,10 @@ org.ow2.asm asm-commons + + it.unimi.dsi + fastutil + diff --git a/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java b/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java new file mode 100644 index 000000000000..c45f5787742d --- /dev/null +++ b/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java @@ -0,0 +1,228 @@ +/* + * 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.output; + +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 java.io.IOException; +import java.io.InputStream; +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 ByteBufferOutputBytes extends OutputBytes +{ + static final int BUFFER_SIZE = 64 * 1024; + + final ArrayList buffers = new ArrayList<>(); + int headBufferIndex; + ByteBuffer headBuffer; + long size; + long capacity; + + ByteBufferOutputBytes() + { + 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 OutputBytes so far and writes them into the given ByteBuffer. This method + * changes the position of the out buffer by the {@link #size()} of this OutputBytes. + * + * @throws java.nio.BufferOverflowException if the {@link ByteBuffer#remaining()} capacity of the given buffer is + * smaller than the size of this OutputBytes + */ + 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 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/output/DirectByteBufferOutputBytes.java b/processing/src/main/java/io/druid/output/DirectByteBufferOutputBytes.java new file mode 100644 index 000000000000..29c30da51c1b --- /dev/null +++ b/processing/src/main/java/io/druid/output/DirectByteBufferOutputBytes.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.output; + +import io.druid.java.util.common.ByteBufferUtils; + +import java.nio.ByteBuffer; + +final class DirectByteBufferOutputBytes extends ByteBufferOutputBytes +{ + 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/output/FileOutputBytes.java b/processing/src/main/java/io/druid/output/FileOutputBytes.java new file mode 100644 index 000000000000..77898890707d --- /dev/null +++ b/processing/src/main/java/io/druid/output/FileOutputBytes.java @@ -0,0 +1,135 @@ +/* + * 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.output; + +import com.google.common.io.ByteStreams; +import io.druid.io.Channels; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; + +final class FileOutputBytes extends OutputBytes +{ + 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 + + FileOutputBytes(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 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/output/HeapByteBufferOutputBytes.java b/processing/src/main/java/io/druid/output/HeapByteBufferOutputBytes.java new file mode 100644 index 000000000000..163f1cfd14b5 --- /dev/null +++ b/processing/src/main/java/io/druid/output/HeapByteBufferOutputBytes.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.output; + +import java.nio.ByteBuffer; + +public final class HeapByteBufferOutputBytes extends ByteBufferOutputBytes +{ + @Override + protected ByteBuffer allocateBuffer() + { + return ByteBuffer.allocate(BUFFER_SIZE); + } +} diff --git a/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMedium.java b/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMedium.java new file mode 100644 index 000000000000..1583e2bf0ab0 --- /dev/null +++ b/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMedium.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.output; + +import io.druid.java.util.common.io.Closer; + +import java.io.IOException; + +public final class OffHeapMemoryOutputMedium implements OutputMedium +{ + private final Closer closer = Closer.create(); + + @Override + public OutputBytes makeOutputBytes() + { + DirectByteBufferOutputBytes outputBytes = new DirectByteBufferOutputBytes(); + closer.register(outputBytes::free); + return outputBytes; + } + + @Override + public Closer getCloser() + { + return closer; + } + + @Override + public void close() throws IOException + { + closer.close(); + } +} diff --git a/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMediumFactory.java b/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMediumFactory.java new file mode 100644 index 000000000000..49fda6e1b6ee --- /dev/null +++ b/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMediumFactory.java @@ -0,0 +1,44 @@ +/* + * 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.output; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.File; +import java.io.IOException; + +public class OffHeapMemoryOutputMediumFactory implements OutputMediumFactory +{ + private static final OffHeapMemoryOutputMediumFactory INSTANCE = new OffHeapMemoryOutputMediumFactory(); + + @JsonCreator + public static OffHeapMemoryOutputMediumFactory instance() + { + return INSTANCE; + } + + private OffHeapMemoryOutputMediumFactory() {} + + @Override + public OutputMedium makeOutputMedium(File outDir) throws IOException + { + return new OffHeapMemoryOutputMedium(); + } +} diff --git a/processing/src/main/java/io/druid/output/OutputBytes.java b/processing/src/main/java/io/druid/output/OutputBytes.java new file mode 100644 index 000000000000..6d3b354ac7af --- /dev/null +++ b/processing/src/main/java/io/druid/output/OutputBytes.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.output; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +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. + * + * OutputBytes is a resource that is managed by {@link OutputMedium}, so it's own {@link #close()} method does nothing. + * However OutputBytes should appear closed, i. e. {@link #isOpen()} returns false, after the parental OutputMedium is + * closed. + */ +public abstract class OutputBytes 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 OutputBytes so far. + */ + public abstract long size() throws IOException; + + /** + * Takes all bytes that are written to this OutputBytes 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 OutputBytes so far. The returned + * InputStream must be closed properly after it's used up. + */ + public abstract InputStream asInputStream() throws IOException; + + /** + * @deprecated does nothing. + */ + @Deprecated + @Override + public final void close() + { + // Does nothing. + } +} diff --git a/processing/src/main/java/io/druid/output/OutputMedium.java b/processing/src/main/java/io/druid/output/OutputMedium.java new file mode 100644 index 000000000000..3b2978b60b5e --- /dev/null +++ b/processing/src/main/java/io/druid/output/OutputMedium.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.output; + +import io.druid.java.util.common.io.Closer; + +import java.io.Closeable; +import java.io.IOException; + +/** + * OutputMedium manages the resources of a bunch of {@link OutputBytes} objects of the same kind, created by calling + * {@link #makeOutputBytes()} on the OutputMedium object. When OutputMedium is closed, all child OutputBytes couldn't + * be used anymore. + */ +public interface OutputMedium extends Closeable +{ + /** + * Creates a new empty {@link OutputBytes}, attached to this OutputMedium. When this OutputMedium is closed, the + * returned OutputBytes couldn't be used anymore. + */ + OutputBytes makeOutputBytes() throws IOException; + + /** + * Returns a closer of this OutputMedium, which is closed in this OutputMedium's close() method. Could be used to + * "attach" some random resources to this OutputMedium, to be closed at the same time. + */ + Closer getCloser(); +} diff --git a/processing/src/main/java/io/druid/output/OutputMediumFactory.java b/processing/src/main/java/io/druid/output/OutputMediumFactory.java new file mode 100644 index 000000000000..4e7da59e512c --- /dev/null +++ b/processing/src/main/java/io/druid/output/OutputMediumFactory.java @@ -0,0 +1,48 @@ +/* + * 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.output; + +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 = TmpFileOutputMediumFactory.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "tmpFile", value = TmpFileOutputMediumFactory.class), + @JsonSubTypes.Type(name = "offHeapMemory", value = OffHeapMemoryOutputMediumFactory.class), +}) +public interface OutputMediumFactory +{ + static Set builtInFactories() + { + return ImmutableSet.of(TmpFileOutputMediumFactory.instance(), OffHeapMemoryOutputMediumFactory.instance()); + } + + /** + * Creates a new OutputMedium. If this type of OutputMedium 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. + */ + OutputMedium makeOutputMedium(File outDir) throws IOException; +} diff --git a/processing/src/main/java/io/druid/output/OutputMediumModule.java b/processing/src/main/java/io/druid/output/OutputMediumModule.java new file mode 100644 index 000000000000..d0f7cc8af98f --- /dev/null +++ b/processing/src/main/java/io/druid/output/OutputMediumModule.java @@ -0,0 +1,33 @@ +/* + * 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.output; + +import com.google.inject.Binder; +import com.google.inject.Module; +import io.druid.guice.JsonConfigProvider; + +public class OutputMediumModule implements Module +{ + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.defaultOutputMediumFactory", OutputMediumFactory.class); + } +} diff --git a/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java b/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java new file mode 100644 index 000000000000..38c3d9338f1c --- /dev/null +++ b/processing/src/main/java/io/druid/output/TmpFileOutputMedium.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.output; + +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 TmpFileOutputMedium implements OutputMedium +{ + private final File dir; + private final Closer closer = Closer.create(); + + TmpFileOutputMedium(File outDir) throws IOException + { + File tmpOutputFilesDir = new File(outDir, "tmpOutputFiles"); + FileUtils.forceMkdir(tmpOutputFilesDir); + closer.register(() -> FileUtils.deleteDirectory(tmpOutputFilesDir)); + this.dir = tmpOutputFilesDir; + } + + @Override + public OutputBytes makeOutputBytes() throws IOException + { + File file = File.createTempFile("filePeon", null, dir); + FileChannel ch = FileChannel.open( + file.toPath(), + StandardOpenOption.READ, + StandardOpenOption.WRITE, + StandardOpenOption.DELETE_ON_CLOSE + ); + closer.register(ch); + return new FileOutputBytes(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/output/TmpFileOutputMediumFactory.java b/processing/src/main/java/io/druid/output/TmpFileOutputMediumFactory.java new file mode 100644 index 000000000000..ac8b434ab37e --- /dev/null +++ b/processing/src/main/java/io/druid/output/TmpFileOutputMediumFactory.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.output; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.File; +import java.io.IOException; + +public final class TmpFileOutputMediumFactory implements OutputMediumFactory +{ + private static final TmpFileOutputMediumFactory INSTANCE = new TmpFileOutputMediumFactory(); + + @JsonCreator + public static TmpFileOutputMediumFactory instance() + { + return INSTANCE; + } + + private TmpFileOutputMediumFactory() {} + @Override + public OutputMedium makeOutputMedium(File outDir) throws IOException + { + return new TmpFileOutputMedium(outDir); + } +} 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 26fda75b9aa3..3a255ff45459 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 io.druid.data.input.InputRow; import io.druid.hll.HyperLogLogCollector; import io.druid.hll.HyperLogLogHash; +import io.druid.output.OutputMedium; 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 int compare(HyperLogLogCollector o1, HyperLogLogCollector o2) } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String column) + public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, 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 e8c7e2e95d1a..761a2cfb95ab 100644 --- a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java @@ -19,25 +19,29 @@ 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.Closer; +import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; 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.IndexedIntsIterator; 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.IntIterator; +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 - @@ -57,7 +61,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()) { @@ -127,14 +131,16 @@ public static CompressedVSizeIndexedSupplier fromIterable( offsetMax, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(offsetMax), byteOrder, - compression + compression, + closer ); CompressedVSizeIntsIndexedSupplier valuesSupplier = CompressedVSizeIntsIndexedSupplier.fromList( values, maxValue, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), byteOrder, - compression + compression, + closer ); return new CompressedVSizeIndexedSupplier(headerSupplier, valuesSupplier); } diff --git a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java index 26e10afa5543..1c1ebd080164 100644 --- a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java @@ -19,22 +19,24 @@ package io.druid.segment; +import com.google.common.annotations.VisibleForTesting; import io.druid.java.util.common.IAE; +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.data.CompressedIntsIndexedSupplier; -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.IndexedMultivalue; import io.druid.segment.data.WritableSupplier; +import it.unimi.dsi.fastutil.ints.IntArrayList; 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; /** * The format is mostly the same with CompressedVSizeIndexedSupplier(which has version 0x2, so we call it V2), @@ -50,7 +52,7 @@ public class CompressedVSizeIndexedV3Supplier 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 +111,32 @@ public static CompressedVSizeIndexedV3Supplier fromIterable( offsetList, offsetChunkFactor, byteOrder, - compression + compression, + closer ); CompressedVSizeIntsIndexedSupplier valuesSupplier = CompressedVSizeIntsIndexedSupplier.fromList( values, maxValue, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), byteOrder, - compression + compression, + closer ); return new CompressedVSizeIndexedV3Supplier(headerSupplier, valuesSupplier); } @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { return 1 + offsetSupplier.getSerializedSize() + valueSupplier.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel) throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { channel.write(ByteBuffer.wrap(new byte[]{VERSION})); - offsetSupplier.writeToChannel(channel); - valueSupplier.writeToChannel(channel); + offsetSupplier.writeTo(channel, smoosher); + valueSupplier.writeTo(channel, smoosher); } @Override diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java index f0c24cd80794..f304b4f8dc26 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -19,13 +19,12 @@ package io.druid.segment; +import io.druid.output.OutputMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; import java.io.Closeable; -import java.io.File; import java.io.IOException; /** @@ -86,8 +85,6 @@ public interface DimensionHandler * See {@link DimensionMergerV9} interface for more information. * * @param indexSpec Specification object for the index merge - * @param outDir Location to store files generated by the merging process - * @param ioPeon ioPeon object passed in by IndexMerger, manages files created by the merging process * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler * @param progress ProgressIndicator used by the merging process @@ -95,8 +92,7 @@ public interface DimensionHandler */ DimensionMergerV9 makeMerger( IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + OutputMedium outputMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) throws IOException; diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java index 0e74e23990ac..7ea1ddd954ec 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.output.OutputMedium; 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 @@ public class DoubleColumnSerializer implements GenericColumnSerializer { public static DoubleColumnSerializer create( - IOPeon ioPeon, + OutputMedium outputMedium, String filenameBase, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - return new DoubleColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression); + return new DoubleColumnSerializer(outputMedium, filenameBase, IndexIO.BYTE_ORDER, compression); } - private final IOPeon ioPeon; + private final OutputMedium outputMedium; 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( + OutputMedium outputMedium, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -65,7 +64,7 @@ public DoubleColumnSerializer( public void open() throws IOException { writer = CompressionFactory.getDoubleSerializer( - ioPeon, + outputMedium, StringUtils.format("%s.double_column", filenameBase), byteOrder, compression @@ -81,21 +80,15 @@ public void serialize(Object obj) throws IOException } @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 a6b2384e424e..b1491d3d2a56 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java @@ -19,15 +19,14 @@ package io.druid.segment; +import io.druid.output.OutputMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.DoubleColumn; import io.druid.segment.column.GenericColumn; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; import java.io.Closeable; -import java.io.File; import java.io.IOException; public class DoubleDimensionHandler implements DimensionHandler @@ -53,14 +52,16 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress + IndexSpec indexSpec, + OutputMedium outputMedium, + ColumnCapabilities capabilities, + ProgressIndicator progress ) throws IOException { return new DoubleDimensionMergerV9( dimensionName, indexSpec, - outDir, - ioPeon, + outputMedium, capabilities, progress ); diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java index 87520d41ac8d..26e5cff9de91 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java @@ -20,14 +20,13 @@ package io.druid.segment; import io.druid.java.util.common.io.Closer; +import io.druid.output.OutputMedium; import io.druid.segment.column.ColumnCapabilities; 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 java.io.File; import java.io.IOException; import java.nio.IntBuffer; import java.util.List; @@ -38,15 +37,12 @@ public class DoubleDimensionMergerV9 implements DimensionMergerV9 protected ProgressIndicator progress; protected final IndexSpec indexSpec; protected ColumnCapabilities capabilities; - protected final File outDir; - protected IOPeon ioPeon; private DoubleColumnSerializer serializer; public DoubleDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + OutputMedium outputMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -54,40 +50,23 @@ public DoubleDimensionMergerV9( this.dimensionName = dimensionName; this.indexSpec = indexSpec; this.capabilities = capabilities; - this.outDir = outDir; - this.ioPeon = ioPeon; this.progress = progress; try { - setupEncodedValueWriter(); + setupEncodedValueWriter(outputMedium); } catch (IOException ioe) { throw new RuntimeException(ioe); } } - protected void setupEncodedValueWriter() throws IOException + protected void setupEncodedValueWriter(OutputMedium outputMedium) throws IOException { - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = DoubleColumnSerializer.create(ioPeon, dimensionName, metCompression); + final CompressionStrategy metCompression = indexSpec.getMetricCompression(); + this.serializer = DoubleColumnSerializer.create(outputMedium, 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 { @@ -118,4 +97,18 @@ public boolean canSkip() // 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 a62eb697227f..efbd0c71c1e5 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSelector.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSelector.java @@ -22,11 +22,6 @@ import io.druid.query.monomorphicprocessing.CalledFromHotLoop; import io.druid.query.monomorphicprocessing.HotLoopCallee; -/** - * An object that gets a metric value. Metric values are always floats and there is an assumption that the - * FloatColumnSelector has a handle onto some other stateful object (e.g. an Offset) which is changing between calls - * to get() (though, that doesn't have to be the case if you always want the same value...). - */ public interface FloatColumnSelector extends ColumnValueSelector, HotLoopCallee { @CalledFromHotLoop diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java index 15560f14d855..abc5dffae20b 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 @@ 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.output.OutputMedium; 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 @@ public class FloatColumnSerializer implements GenericColumnSerializer { public static FloatColumnSerializer create( - IOPeon ioPeon, + OutputMedium outputMedium, String filenameBase, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - return new FloatColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression); + return new FloatColumnSerializer(outputMedium, filenameBase, IndexIO.BYTE_ORDER, compression); } - private final IOPeon ioPeon; + private final OutputMedium outputMedium; 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( + OutputMedium outputMedium, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -64,7 +64,7 @@ public FloatColumnSerializer( public void open() throws IOException { writer = CompressionFactory.getFloatSerializer( - ioPeon, + outputMedium, StringUtils.format("%s.float_column", filenameBase), byteOrder, compression @@ -80,21 +80,14 @@ public void serialize(Object obj) throws IOException } @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 f95b0d96534d..47385065b5c5 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java @@ -19,15 +19,14 @@ package io.druid.segment; +import io.druid.output.OutputMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; -import io.druid.segment.column.GenericColumn; import io.druid.segment.column.FloatColumn; -import io.druid.segment.data.IOPeon; +import io.druid.segment.column.GenericColumn; import io.druid.segment.data.Indexed; import java.io.Closeable; -import java.io.File; import java.io.IOException; public class FloatDimensionHandler implements DimensionHandler @@ -53,14 +52,16 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress + IndexSpec indexSpec, + OutputMedium outputMedium, + ColumnCapabilities capabilities, + ProgressIndicator progress ) throws IOException { return new FloatDimensionMergerV9( dimensionName, indexSpec, - outDir, - ioPeon, + outputMedium, capabilities, progress ); diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java index d65da367dc2c..fb9bfe6b45b5 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java @@ -20,14 +20,13 @@ package io.druid.segment; import io.druid.java.util.common.io.Closer; +import io.druid.output.OutputMedium; import io.druid.segment.column.ColumnCapabilities; 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 java.io.File; import java.io.IOException; import java.nio.IntBuffer; import java.util.List; @@ -38,16 +37,13 @@ public class FloatDimensionMergerV9 implements DimensionMergerV9 protected ProgressIndicator progress; protected final IndexSpec indexSpec; protected ColumnCapabilities capabilities; - protected final File outDir; - protected IOPeon ioPeon; private FloatColumnSerializer serializer; public FloatDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + OutputMedium outputMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -55,22 +51,20 @@ public FloatDimensionMergerV9( this.dimensionName = dimensionName; this.indexSpec = indexSpec; this.capabilities = capabilities; - this.outDir = outDir; - this.ioPeon = ioPeon; this.progress = progress; try { - setupEncodedValueWriter(); + setupEncodedValueWriter(outputMedium); } catch (IOException ioe) { throw new RuntimeException(ioe); } } - protected void setupEncodedValueWriter() throws IOException + private void setupEncodedValueWriter(OutputMedium outputMedium) throws IOException { - final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = FloatColumnSerializer.create(ioPeon, dimensionName, metCompression); + final CompressionStrategy metCompression = indexSpec.getMetricCompression(); + this.serializer = FloatColumnSerializer.create(outputMedium, dimensionName, metCompression); serializer.open(); } @@ -108,7 +102,6 @@ public boolean canSkip() @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 474d7dafe524..95215a5ec175 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 { public void open() throws IOException; public void serialize(Object obj) throws IOException; - - public long getSerializedSize(); - - public 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 494b556fd3c1..970d42a5178d 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.io.smoosh.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OutputMediumFactory; 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.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 OutputMediumFactory defaultOutputMediumFactory; @Inject - public IndexIO(ObjectMapper mapper, ColumnConfig columnConfig) + public IndexIO(ObjectMapper mapper, OutputMediumFactory defaultOutputMediumFactory, ColumnConfig columnConfig) { this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); + this.defaultOutputMediumFactory = Preconditions.checkNotNull(defaultOutputMediumFactory, "null OutputMediumFactory"); Preconditions.checkNotNull(columnConfig, "null ColumnConfig"); ImmutableMap.Builder indexLoadersBuilder = ImmutableMap.builder(); LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig); @@ -224,13 +228,17 @@ public boolean convertSegment( File converted, IndexSpec indexSpec, boolean forceIfCurrent, - boolean validate + boolean validate, + @Nullable OutputMediumFactory outputMediumFactory ) 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 (outputMediumFactory == null) { + outputMediumFactory = this.defaultOutputMediumFactory; + } + new IndexMergerV9(mapper, this, outputMediumFactory).convert(toConvert, converted, indexSpec); if (validate) { validateTwoSegments(toConvert, converted); } @@ -477,7 +485,7 @@ public QueryableIndex load(File inDir, ObjectMapper mapper) throws IOException metric, new ColumnBuilder() .setType(ValueType.FLOAT) - .setGenericColumn(new FloatGenericColumnSupplier(metricHolder.floatType, BYTE_ORDER)) + .setGenericColumn(new FloatGenericColumnSupplier(metricHolder.floatType)) .build() ); } else if (metricHolder.getType() == MetricHolder.MetricType.COMPLEX) { diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index 7e75122cfba3..f5b6c58505b3 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; @@ -36,6 +37,7 @@ import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.CloseableIterator; +import io.druid.output.OutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.data.Indexed; @@ -157,7 +159,12 @@ static > ArrayList mergeIndexed(List> ArrayList mergeIndexed(List indexes, boolean rollup, @@ -206,30 +223,22 @@ File merge( IndexSpec indexSpec ) throws IOException; - File merge( - List indexes, - boolean rollup, - AggregatorFactory[] metricAggs, + File convert(File inDir, File outDir, IndexSpec indexSpec) throws IOException; + + File convert( + File inDir, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + ProgressIndicator progress, + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException; - // 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 append(List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec) - throws IOException; - File append( List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException; interface IndexSeeker @@ -411,7 +420,8 @@ public Rowboat apply(Rowboat lhs, Rowboat rhs) 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 af59c1522d7e..b4641e5f90b8 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.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; @@ -46,18 +45,18 @@ import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedWriter; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OutputMedium; +import io.druid.output.OutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.Column; 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; @@ -75,7 +74,6 @@ 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 @@ 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 OutputMediumFactory defaultOutputMediumFactory; @Inject - public IndexMergerV9( - ObjectMapper mapper, - IndexIO indexIO - ) + public IndexMergerV9(ObjectMapper mapper, IndexIO indexIO, OutputMediumFactory defaultOutputMediumFactory) { 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.defaultOutputMediumFactory = + Preconditions.checkNotNull(defaultOutputMediumFactory, "null OutputMediumFactory"); } private File makeIndexFiles( @@ -125,7 +111,8 @@ private File makeIndexFiles( final List mergedDimensions, final List mergedMetrics, final Function>, Iterable> rowMergerFn, - final IndexSpec indexSpec + final IndexSpec indexSpec, + final @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { progress.start(); @@ -163,21 +150,14 @@ public Metadata apply(IndexableAdapter input) 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); - - File tmpPeonFilesDir = new File(v9TmpDir, "tmpPeonFiles"); - FileUtils.forceMkdir(tmpPeonFilesDir); - registerDeleteDirectory(closer, tmpPeonFilesDir); - final IOPeon ioPeon = new TmpFileIOPeon(tmpPeonFilesDir, false); - closer.register(ioPeon); + FileUtils.forceMkdir(outDir); + + OutputMediumFactory omf = outputMediumFactory != null ? outputMediumFactory : defaultOutputMediumFactory; + log.info("Using OutputMediumFactory[%s]", omf.getClass().getSimpleName()); + OutputMedium outputMedium = omf.makeOutputMedium(outDir); + closer.register(outputMedium); 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,18 +176,13 @@ public Metadata apply(IndexableAdapter input) 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, outputMedium, dimCapabilities.get(i), progress)); } /************* Setup Dim Conversions **************/ progress.progress(); startTime = System.currentTimeMillis(); - final ArrayList> dimConversions = Lists.newArrayListWithCapacity(adapters.size()); - final ArrayList dimensionSkipFlag = Lists.newArrayListWithCapacity(mergedDimensions.size()); - final ArrayList convertMissingDimsFlags = Lists.newArrayListWithCapacity(mergedDimensions.size()); - writeDimValueAndSetupDimConversion( - adapters, progress, mergedDimensions, mergers - ); + writeDimValueAndSetupDimConversion(adapters, progress, mergedDimensions, mergers); log.info("Completed dim conversions in %,d millis.", System.currentTimeMillis() - startTime); /************* Walk through data sets, merge them, and write merged columns *************/ @@ -221,15 +196,17 @@ public Metadata apply(IndexableAdapter input) handlers, mergers ); - final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon, indexSpec); + final LongColumnSerializer timeWriter = setupTimeWriter(outputMedium, indexSpec); final ArrayList metWriters = setupMetricsWriters( - ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec + outputMedium, + 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"; @@ -251,9 +228,7 @@ public Metadata apply(IndexableAdapter input) /************* 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(); @@ -318,8 +293,8 @@ private void makeIndexBinary( + 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; @@ -333,9 +308,7 @@ private void makeIndexBinary( 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")); @@ -361,7 +334,6 @@ private void makeMetricsColumns( 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); @@ -427,8 +399,6 @@ private void makeTimeColumn( progress.startSection(section); long startTime = System.currentTimeMillis(); - timeWriter.close(); - final ColumnDescriptor serdeficator = ColumnDescriptor .builder() .setValueType(ValueType.LONG) @@ -453,10 +423,11 @@ private void makeColumn( 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); } } @@ -527,10 +498,12 @@ private void mergeIndexesAndWriteColumns( progress.stopSection(section); } - private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon, final IndexSpec indexSpec) throws IOException + private LongColumnSerializer setupTimeWriter(OutputMedium outputMedium, IndexSpec indexSpec) throws IOException { LongColumnSerializer timeWriter = LongColumnSerializer.create( - ioPeon, "little_end_time", CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY, + outputMedium, + "little_end_time", + CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY, indexSpec.getLongEncoding() ); // we will close this writer after we added all the timestamps @@ -539,7 +512,7 @@ private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon, final IndexSpe } private ArrayList setupMetricsWriters( - final IOPeon ioPeon, + final OutputMedium outputMedium, final List mergedMetrics, final Map metricsValueTypes, final Map metricTypeNames, @@ -547,20 +520,20 @@ private ArrayList setupMetricsWriters( ) 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(outputMedium, metric, metCompression, longEncoding); break; case FLOAT: - writer = FloatColumnSerializer.create(ioPeon, metric, metCompression); + writer = FloatColumnSerializer.create(outputMedium, metric, metCompression); break; case DOUBLE: - writer = DoubleColumnSerializer.create(ioPeon, metric, metCompression); + writer = DoubleColumnSerializer.create(outputMedium, metric, metCompression); break; case COMPLEX: final String typeName = metricTypeNames.get(metric); @@ -568,7 +541,7 @@ private ArrayList setupMetricsWriters( if (serde == null) { throw new ISE("Unknown type[%s]", typeName); } - writer = serde.getSerializer(ioPeon, metric); + writer = serde.getSerializer(outputMedium, metric); break; default: throw new ISE("Unknown type[%s]", type); @@ -634,10 +607,11 @@ private void mergeCapabilities( public File persist( final IncrementalIndex index, File outDir, - IndexSpec indexSpec + IndexSpec indexSpec, + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { - return persist(index, index.getInterval(), outDir, indexSpec); + return persist(index, index.getInterval(), outDir, indexSpec, outputMediumFactory); } @Override @@ -645,10 +619,11 @@ public File persist( final IncrementalIndex index, final Interval dataInterval, File outDir, - IndexSpec indexSpec + IndexSpec indexSpec, + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { - return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator()); + return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator(), outputMediumFactory); } @Override @@ -657,7 +632,8 @@ public File persist( final Interval dataInterval, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + ProgressIndicator progress, + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { if (index.isEmpty()) { @@ -694,7 +670,8 @@ public File persist( index.getMetricAggs(), outDir, indexSpec, - progress + progress, + outputMediumFactory ); } @@ -704,10 +681,19 @@ public File mergeQueryableIndex( boolean rollup, final AggregatorFactory[] metricAggs, File outDir, - IndexSpec indexSpec + IndexSpec indexSpec, + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { - return mergeQueryableIndex(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); + return mergeQueryableIndex( + indexes, + rollup, + metricAggs, + outDir, + indexSpec, + new BaseProgressIndicator(), + outputMediumFactory + ); } @Override @@ -717,7 +703,8 @@ public File mergeQueryableIndex( final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + ProgressIndicator progress, + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { return merge( @@ -726,7 +713,8 @@ public File mergeQueryableIndex( metricAggs, outDir, indexSpec, - progress + progress, + outputMediumFactory ); } @@ -739,17 +727,17 @@ public File merge( IndexSpec indexSpec ) throws IOException { - return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator()); + return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator(), null); } - @Override - public File merge( + private File merge( List indexes, final boolean rollup, final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + ProgressIndicator progress, + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { FileUtils.deleteDirectory(outDir); @@ -854,19 +842,25 @@ public int compare(Rowboat left, Rowboat right) mergedDimensions, mergedMetrics, rowMergerFn, - indexSpec + indexSpec, + outputMediumFactory ); } @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(), defaultOutputMediumFactory); } @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 OutputMediumFactory outputMediumFactory + ) throws IOException { try (QueryableIndex index = indexIO.loadIndex(inDir)) { final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index); @@ -886,26 +880,19 @@ public Iterable apply(ArrayList> input) return input.get(0); } }, - indexSpec + indexSpec, + outputMediumFactory ); } } - @Override - public File append( - List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec - ) throws IOException - { - return append(indexes, aggregators, outDir, indexSpec, new BaseProgressIndicator()); - } - @Override public File append( List indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec, - ProgressIndicator progress + @Nullable OutputMediumFactory outputMediumFactory ) throws IOException { FileUtils.deleteDirectory(outDir); @@ -952,11 +939,12 @@ public Iterable apply( indexes, aggregators, outDir, - progress, + new BaseProgressIndicator(), mergedDimensions, mergedMetrics, rowMergerFn, - indexSpec + indexSpec, + outputMediumFactory ); } diff --git a/processing/src/main/java/io/druid/segment/IndexSpec.java b/processing/src/main/java/io/druid/segment/IndexSpec.java index 67ea64d2ac36..073a6a879af6 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.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 @@ */ 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 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 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 BitmapSerdeFactory getBitmapSerdeFactory() } @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/LongColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java index b31a5f504545..e5f34a6dda06 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 @@ 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.output.OutputMedium; 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 @@ public class LongColumnSerializer implements GenericColumnSerializer { public static LongColumnSerializer create( - IOPeon ioPeon, + OutputMedium outputMedium, String filenameBase, - CompressedObjectStrategy.CompressionStrategy compression, + CompressionStrategy compression, CompressionFactory.LongEncodingStrategy encoding ) { - return new LongColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression, encoding); + return new LongColumnSerializer(outputMedium, filenameBase, IndexIO.BYTE_ORDER, compression, encoding); } - private final IOPeon ioPeon; + private final OutputMedium outputMedium; 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( + OutputMedium outputMedium, String filenameBase, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression, + CompressionStrategy compression, CompressionFactory.LongEncodingStrategy encoding ) { - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -71,7 +71,7 @@ public LongColumnSerializer( public void open() throws IOException { writer = CompressionFactory.getLongSerializer( - ioPeon, + outputMedium, StringUtils.format("%s.long_column", filenameBase), byteOrder, encoding, @@ -88,21 +88,14 @@ public void serialize(Object obj) throws IOException } @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 0511a75df70a..a936244c4f61 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java @@ -19,15 +19,14 @@ package io.druid.segment; +import io.druid.output.OutputMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.GenericColumn; import io.druid.segment.column.LongColumn; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; import java.io.Closeable; -import java.io.File; import java.io.IOException; public class LongDimensionHandler implements DimensionHandler @@ -53,14 +52,16 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( - IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress + IndexSpec indexSpec, + OutputMedium outputMedium, + ColumnCapabilities capabilities, + ProgressIndicator progress ) throws IOException { return new LongDimensionMergerV9( dimensionName, indexSpec, - outDir, - ioPeon, + outputMedium, capabilities, progress ); diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java index c1afc2f67467..f6891d87ea0d 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java @@ -21,15 +21,14 @@ import com.google.common.base.Throwables; import io.druid.java.util.common.io.Closer; +import io.druid.output.OutputMedium; import io.druid.segment.column.ColumnCapabilities; 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 java.io.File; import java.io.IOException; import java.nio.IntBuffer; import java.util.List; @@ -40,15 +39,13 @@ public class LongDimensionMergerV9 implements DimensionMergerV9 protected ProgressIndicator progress; protected final IndexSpec indexSpec; protected ColumnCapabilities capabilities; - protected final File outDir; - protected IOPeon ioPeon; + private final OutputMedium outputMedium; protected LongColumnSerializer serializer; - public LongDimensionMergerV9( + LongDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + OutputMedium outputMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -56,8 +53,7 @@ public LongDimensionMergerV9( this.dimensionName = dimensionName; this.indexSpec = indexSpec; this.capabilities = capabilities; - this.outDir = outDir; - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; this.progress = progress; try { @@ -70,9 +66,9 @@ public LongDimensionMergerV9( protected void setupEncodedValueWriter() 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(outputMedium, dimensionName, metCompression, longEncoding); serializer.open(); } @@ -110,7 +106,6 @@ public boolean canSkip() @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 97755338e024..9e0bff9e171f 100644 --- a/processing/src/main/java/io/druid/segment/MetricHolder.java +++ b/processing/src/main/java/io/druid/segment/MetricHolder.java @@ -19,33 +19,19 @@ package io.druid.segment; -import com.google.common.io.ByteSink; -import com.google.common.io.ByteStreams; -import com.google.common.io.InputSupplier; -import com.google.common.io.OutputSupplier; 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.io.smoosh.SmooshedFileMapper; import io.druid.segment.data.CompressedDoublesIndexedSupplier; import io.druid.segment.data.CompressedFloatsIndexedSupplier; import io.druid.segment.data.CompressedLongsIndexedSupplier; -import io.druid.segment.data.DoubleSupplierSerializer; -import io.druid.segment.data.FloatSupplierSerializer; import io.druid.segment.data.GenericIndexed; -import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.Indexed; -import io.druid.segment.data.IndexedDoubles; -import io.druid.segment.data.IndexedFloats; -import io.druid.segment.data.IndexedLongs; -import io.druid.segment.data.LongSupplierSerializer; import io.druid.segment.data.ObjectStrategy; import io.druid.segment.serde.ComplexMetricSerde; import io.druid.segment.serde.ComplexMetrics; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; @@ -56,65 +42,6 @@ public class MetricHolder private static final byte[] version = new byte[]{0x0}; private static final SerializerUtils serializerUtils = new SerializerUtils(); - public static MetricHolder floatMetric(String name, CompressedFloatsIndexedSupplier column) - { - MetricHolder retVal = new MetricHolder(name, "float"); - retVal.floatType = column; - return retVal; - } - - public static MetricHolder complexMetric(String name, String typeName, Indexed column) - { - MetricHolder retVal = new MetricHolder(name, typeName); - retVal.complexType = column; - return retVal; - } - - public static void writeComplexMetric( - OutputSupplier outSupplier, String name, String typeName, GenericIndexedWriter column - ) throws IOException - { - try (OutputStream out = outSupplier.getOutput()) { - out.write(version); - serializerUtils.writeString(out, name); - serializerUtils.writeString(out, typeName); - final InputSupplier supplier = column.combineStreams(); - try (InputStream in = supplier.getInput()) { - ByteStreams.copy(in, out); - } - } - } - - public static void writeFloatMetric( - final ByteSink outSupplier, String name, FloatSupplierSerializer column - ) throws IOException - { - outSupplier.write(version); - serializerUtils.writeString(toOutputSupplier(outSupplier), name); - serializerUtils.writeString(toOutputSupplier(outSupplier), "float"); - column.closeAndConsolidate(outSupplier); - } - - public static void writeLongMetric( - ByteSink outSupplier, String name, LongSupplierSerializer column - ) throws IOException - { - outSupplier.write(version); - serializerUtils.writeString(toOutputSupplier(outSupplier), name); - serializerUtils.writeString(toOutputSupplier(outSupplier), "long"); - column.closeAndConsolidate(outSupplier); - } - - public static void writeDoubleMetric(ByteSink outSupplier, String name, DoubleSupplierSerializer column - ) throws IOException - { - outSupplier.write(version); - serializerUtils.writeString(toOutputSupplier(outSupplier), name); - serializerUtils.writeString(toOutputSupplier(outSupplier), "double"); - column.closeAndConsolidate(outSupplier); - } - - public static MetricHolder fromByteBuffer(ByteBuffer buf, SmooshedFileMapper mapper) throws IOException { return fromByteBuffer(buf, null, mapper); @@ -137,10 +64,10 @@ public static MetricHolder fromByteBuffer(ByteBuffer buf, ObjectStrategy strateg holder.longType = CompressedLongsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder(), mapper); break; case FLOAT: - holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder(), mapper); + holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); break; case DOUBLE: - holder.doubleType = CompressedDoublesIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder(), mapper); + holder.doubleType = CompressedDoublesIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder()); break; case COMPLEX: if (strategy != null) { @@ -160,19 +87,6 @@ public static MetricHolder fromByteBuffer(ByteBuffer buf, ObjectStrategy strateg return holder; } - // This is only for guava14 compat. Eventually it should be able to be removed. - private static OutputSupplier toOutputSupplier(final ByteSink sink) - { - return new OutputSupplier() - { - @Override - public OutputStream getOutput() throws IOException - { - return sink.openStream(); - } - }; - } - private final String name; private final String typeName; private final MetricType type; @@ -227,34 +141,4 @@ public MetricType getType() return type; } - public IndexedLongs getLongType() - { - assertType(MetricType.LONG); - return longType.get(); - } - - public IndexedFloats getFloatType() - { - assertType(MetricType.FLOAT); - return floatType.get(); - } - - public IndexedDoubles getDoubleType() - { - assertType(MetricType.DOUBLE); - return doubleType.get(); - } - - public Indexed getComplexType() - { - assertType(MetricType.COMPLEX); - return complexType; - } - - private void assertType(MetricType type) - { - if (this.type != type) { - throw new IAE("type[%s] cannot be cast to [%s]", typeName, type); - } - } } diff --git a/processing/src/main/java/io/druid/segment/Rowboat.java b/processing/src/main/java/io/druid/segment/Rowboat.java index 1ef2dbf3aa19..c047fd1c567b 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 Rowboat( 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 bfe510c6f781..85432c7fc7d2 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 @@ import com.google.common.primitives.Ints; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; +import io.druid.output.OutputMedium; 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; @@ -193,13 +192,12 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + OutputMedium outputMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) { - return new StringDimensionMergerV9(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress); + return new StringDimensionMergerV9(dimensionName, indexSpec, outputMedium, capabilities, progress); } } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index 887620c36ac6..555b0fc8a57c 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -20,33 +20,30 @@ package io.druid.segment; import com.google.common.base.Splitter; -import com.google.common.collect.Iterables; +import com.google.common.base.Strings; import com.google.common.collect.Lists; -import com.google.common.io.ByteStreams; -import com.google.common.io.Files; import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.MutableBitmap; import io.druid.collections.spatial.ImmutableRTree; import io.druid.collections.spatial.RTree; import io.druid.collections.spatial.split.LinearGutmanSplitStrategy; -import io.druid.java.util.common.ByteBufferUtils; 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.logger.Logger; +import io.druid.output.OutputMedium; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; import io.druid.segment.data.ArrayIndexed; import io.druid.segment.data.BitmapSerdeFactory; import io.druid.segment.data.ByteBufferWriter; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressedVSizeIndexedV3Writer; import io.druid.segment.data.CompressedVSizeIntsIndexedWriter; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; -import io.druid.segment.data.IOPeon; import io.druid.segment.data.Indexed; import io.druid.segment.data.IndexedInts; import io.druid.segment.data.IndexedIntsWriter; @@ -58,12 +55,8 @@ import it.unimi.dsi.fastutil.ints.IntIterable; import it.unimi.dsi.fastutil.ints.IntIterator; -import java.io.Closeable; -import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.nio.IntBuffer; -import java.nio.MappedByteBuffer; import java.util.ArrayList; import java.util.List; @@ -79,6 +72,9 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 protected String dimensionName; protected GenericIndexedWriter dictionaryWriter; + protected List dictionary; + protected String firstDictionaryValue; + protected int dictionarySize; protected GenericIndexedWriter bitmapWriter; protected ByteBufferWriter spatialWriter; protected ArrayList dimConversions; @@ -86,10 +82,9 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 protected boolean convertMissingValues = false; protected boolean hasNull = false; protected MutableBitmap nullRowsBitmap; - protected IOPeon ioPeon; + private final OutputMedium outputMedium; protected int rowCount = 0; protected ColumnCapabilities capabilities; - protected final File outDir; protected List adapters; protected ProgressIndicator progress; protected final IndexSpec indexSpec; @@ -98,8 +93,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 public StringDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - File outDir, - IOPeon ioPeon, + OutputMedium outputMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -107,8 +101,7 @@ public StringDimensionMergerV9( this.dimensionName = dimensionName; this.indexSpec = indexSpec; this.capabilities = capabilities; - this.outDir = outDir; - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; this.progress = progress; nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } @@ -159,11 +152,13 @@ public void writeMergedValueMetadata(List adapters) throws IOE } String dictFilename = StringUtils.format("%s.dim_values", dimensionName); - dictionaryWriter = new GenericIndexedWriter<>( - ioPeon, - dictFilename, - GenericIndexed.STRING_STRATEGY - ); + dictionaryWriter = new GenericIndexedWriter<>(outputMedium, dictFilename, GenericIndexed.STRING_STRATEGY); + boolean hasSpatial = capabilities.hasSpatialIndexes(); + if (hasSpatial) { + dictionary = new ArrayList<>(); + } + firstDictionaryValue = null; + dictionarySize = 0; dictionaryWriter.open(); cardinality = 0; @@ -171,7 +166,16 @@ public void writeMergedValueMetadata(List adapters) throws IOE dictionaryMergeIterator = new IndexMerger.DictionaryMergeIterator(dimValueLookups, true); while (dictionaryMergeIterator.hasNext()) { - dictionaryWriter.write(dictionaryMergeIterator.next()); + String value = dictionaryMergeIterator.next(); + dictionaryWriter.write(value); + value = Strings.emptyToNull(value); + if (dictionarySize == 0) { + firstDictionaryValue = value; + } + if (hasSpatial) { + dictionary.add(value); + } + dictionarySize++; } for (int i = 0; i < adapters.size(); i++) { @@ -183,6 +187,14 @@ public void writeMergedValueMetadata(List adapters) throws IOE } else if (numMergeIndex == 1) { for (String value : dimValueLookup) { dictionaryWriter.write(value); + value = Strings.emptyToNull(value); + if (dictionarySize == 0) { + firstDictionaryValue = value; + } + if (hasSpatial) { + dictionary.add(value); + } + dictionarySize++; } cardinality = dimValueLookup.size(); } @@ -193,34 +205,37 @@ public void writeMergedValueMetadata(List adapters) throws IOE cardinality, System.currentTimeMillis() - dimStartTime ); - dictionaryWriter.close(); setupEncodedValueWriter(); } 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( + outputMedium, + filenameBase, + cardinality, + compressionStrategy + ); + } else { + encodedValueWriter = new VSizeIndexedWriter(outputMedium, 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( + outputMedium, + filenameBase, + cardinality, + compressionStrategy + ); + } else { + encodedValueWriter = new VSizeIndexedIntsWriter(outputMedium, cardinality); + } } encodedValueWriter.open(); } @@ -279,95 +294,63 @@ public void writeIndexes(List segmentRowNumConversions, Closer closer String bmpFilename = StringUtils.format("%s.inverted", dimensionName); bitmapWriter = new GenericIndexedWriter<>( - ioPeon, + outputMedium, bmpFilename, - bitmapSerdeFactory.getObjectStrategy() + indexSpec.getBitmapSerdeFactory().getObjectStrategy() ); bitmapWriter.open(); - // 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<>( + outputMedium, + new IndexedRTree.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 = new Closeable() - { - @Override - public void close() - { - ByteBufferUtils.unmap(dimValsMapped); - } - }) { - Indexed dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY); - BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory(); - - RTree tree = null; - boolean hasSpatial = capabilities.hasSpatialIndexes(); - if (hasSpatial) { - spatialWriter = new ByteBufferWriter<>( - ioPeon, - StringUtils.format("%s.spatial", dimensionName), - new IndexedRTree.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 < dictionarySize; dictId++) { + progress.progress(); + mergeBitmaps( + segmentRowNumConversions, + bitmapFactory, + tree, + hasSpatial, + dictIdSeeker, + dictId + ); + } - 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)); + } - if (hasSpatial) { - spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree)); - spatialWriter.close(); - } + log.info( + "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", + dimensionName, + dictionarySize, + System.currentTimeMillis() - dimStartTime + ); - log.info( - "Completed dim[%s] inverted with cardinality[%,d] in %,d millis.", - dimensionName, - dimVals.size(), - 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()); @@ -401,14 +384,14 @@ static void mergeBitmaps( 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 = dictionary.get(dictId); if (dimVal != null) { List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); float[] coords = new float[stringCoords.size()]; @@ -431,7 +414,7 @@ public ColumnDescriptor makeColumnDescriptor() { // 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(); @@ -443,7 +426,7 @@ public ColumnDescriptor makeColumnDescriptor() .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 7119012abb83..0793c40f249c 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.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 @@ /** */ -public class ColumnDescriptor +public class ColumnDescriptor implements Serializer { public static Builder builder() { @@ -76,21 +77,21 @@ public List getParts() 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 6045ac9bf489..c144492f05e5 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java @@ -19,54 +19,44 @@ 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.Doubles; import com.google.common.primitives.Ints; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; +import io.druid.io.Channels; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputMedium; import io.druid.segment.CompressedPools; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.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 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( + OutputMedium outputMedium, 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( + outputMedium, + 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 = outputMedium.getCloser(); + this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder); } @Override @@ -86,57 +76,48 @@ public void add(double value) throws IOException { 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 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 metaSize() + flattener.getSerializedSize(); } @Override - public long getSerializedSize() + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - return metaCount + flattener.getSerializedSize(); + writeEndBuffer(); + + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(CompressedDoublesIndexedSupplier.version); + meta.putInt(numInserted); + meta.putInt(CompressedPools.BUFFER_SIZE / Double.BYTES); + meta.put(compression.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + flattener.writeTo(channel, smoosher); } - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private void writeEndBuffer() throws IOException { - try (InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - flattener.writeToChannel(channel, smoosher); + if (endBuffer != null && numInserted > 0) { + endBuffer.flip(); + flattener.write(endBuffer); + endBuffer = null; } } - @Override - public void close() throws IOException + private int metaSize() { - 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(); - } + return 1 + Ints.BYTES + Ints.BYTES + 1; } } 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 ddb112b5063e..f452aa759a02 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,44 @@ 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.io.Channels; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputMedium; import io.druid.segment.CompressedPools; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; +import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.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 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( + OutputMedium outputMedium, 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( + outputMedium, + 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 = outputMedium.getCloser(); + this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder); } @Override @@ -84,58 +75,47 @@ public int size() public void add(float value) throws IOException { if (!endBuffer.hasRemaining()) { - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = FloatBuffer.allocate(sizePer); - endBuffer.mark(); + endBuffer.clear(); + flattener.write(endBuffer); } - - 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 metaSize() + 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(); + writeEndBuffer(); - 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(); - } + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(CompressedFloatsIndexedSupplier.version); + meta.putInt(numInserted); + meta.putInt(CompressedPools.BUFFER_SIZE / Floats.BYTES); + meta.put(compression.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + flattener.writeTo(channel, smoosher); } - @Override - public long getSerializedSize() + private void writeEndBuffer() throws IOException { - return metaCount + flattener.getSerializedSize(); + if (endBuffer != null && numInserted > 0) { + endBuffer.flip(); + flattener.write(endBuffer); + endBuffer = null; + } } - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private int metaSize() { - try (InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - flattener.writeToChannel(channel, smoosher); - } + return 1 + Ints.BYTES + Ints.BYTES + 1; } } 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 495083156406..b8be53cd781d 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedDoubleSupplier.java @@ -20,11 +20,9 @@ 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.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; @@ -42,18 +40,13 @@ public BlockLayoutIndexedDoubleSupplier( int sizePer, ByteBuffer fromBuffer, ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy strategy, - SmooshedFileMapper fileMapper + CompressionStrategy strategy ) { baseDoubleBuffers = GenericIndexed.read( fromBuffer, - VSizeCompressedObjectStrategy.getBufferForOrder(byteOrder, - strategy, - sizePer * Doubles.BYTES - ), - fileMapper + new DecompressingByteBufferObjectStrategy(byteOrder, strategy) ); this.totalSize = totalSize; diff --git a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java index 8ac5b84be10f..43ae7b3d37ac 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutIndexedFloatSupplier.java @@ -20,11 +20,9 @@ package io.druid.segment.data; import com.google.common.base.Supplier; -import com.google.common.primitives.Floats; 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; @@ -41,18 +39,12 @@ public BlockLayoutIndexedFloatSupplier( 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 7202b5bc02f8..54f2ef62d9eb 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 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 BlockLayoutIndexedLongSupplier( 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 abdae5da7a97..1c1cdd635cf2 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java @@ -19,58 +19,43 @@ 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.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputMedium; import io.druid.segment.CompressedPools; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer { - - private final IOPeon ioPeon; + private final OutputMedium outputMedium; 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 ByteOrder byteOrder; + private final CompressionStrategy compression; private int numInserted = 0; + private int numInsertedForNextFlush = 0; private ByteBuffer endBuffer = null; - public BlockLayoutLongSupplierSerializer( - IOPeon ioPeon, + BlockLayoutLongSupplierSerializer( + OutputMedium outputMedium, String filenameBase, - ByteOrder order, + ByteOrder byteOrder, CompressionFactory.LongEncodingWriter writer, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; 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(outputMedium, filenameBase, compression, bufferSize); + this.byteOrder = byteOrder; this.writer = writer; this.compression = compression; } @@ -90,15 +75,18 @@ public int size() @Override public void add(long value) throws IOException { - if (numInserted % sizePer == 0) { + if (numInserted == numInsertedForNextFlush) { + numInsertedForNextFlush += sizePer; if (endBuffer != null) { writer.flush(); - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); + endBuffer.flip(); + flattener.write(endBuffer); + endBuffer.clear(); + } else { + CompressionStrategy.Compressor compressor = compression.getCompressor(); + endBuffer = compressor.allocateInBuffer(writer.getNumBytes(sizePer), outputMedium.getCloser()).order(byteOrder); + writer.setBuffer(endBuffer); } - endBuffer = ByteBuffer.allocate(writer.getNumBytes(sizePer)); - writer.setBuffer(endBuffer); } writer.write(value); @@ -106,50 +94,40 @@ public void add(long value) throws IOException } @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 metaSize() + flattener.getSerializedSize(); } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) 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(); - } + writeEndBuffer(); + + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(CompressedLongsIndexedSupplier.version); + meta.putInt(numInserted); + meta.putInt(sizePer); + writer.putMeta(meta, compression); + meta.flip(); + + Channels.writeFully(channel, meta); + flattener.writeTo(channel, smoosher); } - @Override - public long getSerializedSize() + private void writeEndBuffer() throws IOException { - return metaCount + flattener.getSerializedSize(); + if (endBuffer != null && numInserted > 0) { + writer.flush(); + endBuffer.flip(); + flattener.write(endBuffer); + endBuffer = null; + } } - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private int metaSize() { - try (InputStream meta = ioPeon.makeInputStream(metaFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - flattener.writeToChannel(channel, smoosher); - } + return 1 + Ints.BYTES + Ints.BYTES + writer.metaSize(); } } 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 e0035aeaecdf..3f3f45200fa4 100644 --- a/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferSerializer.java @@ -20,6 +20,7 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; +import io.druid.io.Channels; import java.io.IOException; import java.nio.ByteBuffer; @@ -39,11 +40,12 @@ public static T read(ByteBuffer buffer, ObjectStrategy strategy) return strategy.fromByteBuffer(bufferToUse, size); } - public static void writeToChannel(T obj, ObjectStrategy strategy, WritableByteChannel channel) + public static long writeToChannel(T obj, ObjectStrategy strategy, WritableByteChannel channel) throws IOException { byte[] toWrite = strategy.toBytes(obj); - channel.write(ByteBuffer.allocate(Ints.BYTES).putInt(0, toWrite.length)); - channel.write(ByteBuffer.wrap(toWrite)); + Channels.writeFully(channel, ByteBuffer.wrap(Ints.toByteArray(toWrite.length))); + Channels.writeFully(channel, ByteBuffer.wrap(toWrite)); + return Ints.BYTES + (long) toWrite.length; } } 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 753455888013..5150f6ba0c66 100644 --- a/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java @@ -19,112 +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.output.OutputBytes; +import io.druid.output.OutputMedium; +import io.druid.segment.serde.Serializer; -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 OutputMedium outputMedium; private final ObjectStrategy strategy; - private CountingOutputStream headerOut = null; - private CountingOutputStream valueOut = null; - private final ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); + private OutputBytes headerOut = null; + private OutputBytes valueOut = null; - public ByteBufferWriter( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) + public ByteBufferWriter(OutputMedium outputMedium, ObjectStrategy strategy) { - this.ioPeon = ioPeon; - this.filenameBase = filenameBase; + this.outputMedium = outputMedium; this.strategy = strategy; } public void open() throws IOException { - headerOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("header"))); - valueOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("value"))); + headerOut = outputMedium.makeOutputBytes(); + valueOut = outputMedium.makeOutputBytes(); } public void write(T objectToWrite) throws IOException { - byte[] bytesToWrite = strategy.toBytes(objectToWrite); - SerializerUtils.writeBigEndianIntToOutputStream(headerOut, bytesToWrite.length, helperBuffer); - valueOut.write(bytesToWrite); + long sizeBefore = valueOut.size(); + strategy.writeTo(objectToWrite, valueOut); + headerOut.writeInt(Ints.checkedCast(valueOut.size() - sizeBefore)); } - private String makeFilename(String suffix) + @Override + public long getSerializedSize() throws IOException { - return StringUtils.format("%s.%s", filenameBase, suffix); + return headerOut.size() + valueOut.size(); } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - headerOut.close(); - valueOut.close(); - - final long numBytesWritten = headerOut.getCount() + valueOut.getCount(); + 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, FileSmoosher smoosher) 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 96afb159c784..000000000000 --- 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 5e7778056ba9..000000000000 --- 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 26bbb316e533..a95665e5ea77 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java @@ -22,7 +22,6 @@ import com.google.common.base.Supplier; import com.google.common.primitives.Ints; import io.druid.java.util.common.IAE; -import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import java.io.IOException; import java.nio.ByteBuffer; @@ -38,14 +37,14 @@ public class CompressedDoublesIndexedSupplier implements Supplier supplier; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final CompressionStrategy compression; public CompressedDoublesIndexedSupplier( int totalSize, int sizePer, ByteBuffer buffer, Supplier supplier, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { this.totalSize = totalSize; @@ -61,29 +60,24 @@ public IndexedDoubles get() return supplier.get(); } - public static CompressedDoublesIndexedSupplier fromByteBuffer( - ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper mapper - ) + public static CompressedDoublesIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); 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; if (versionFromBuffer == version) { byte compressionId = buffer.get(); - compression = CompressedObjectStrategy.CompressionStrategy.forId(compressionId); + compression = CompressionStrategy.forId(compressionId); } Supplier supplier = CompressionFactory.getDoubleSupplier( totalSize, sizePer, buffer.asReadOnlyBuffer(), order, - compression, - mapper + compression ); return new CompressedDoublesIndexedSupplier( totalSize, 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 09bb6b5819d8..000000000000 --- 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 839149f5423b..4ee324345a3f 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -20,16 +20,17 @@ 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.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; @@ -38,14 +39,14 @@ public class CompressedFloatsIndexedSupplier implements Supplier 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; @@ -66,43 +67,49 @@ public IndexedFloats get() return supplier.get(); } - public long getSerializedSize() + @Override + public long getSerializedSize() throws IOException { - return buffer.remaining() + 1 + 4 + 4 + 1; + return metaSize() + (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()); + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(version); + meta.putInt(totalSize); + meta.putInt(sizePer); + meta.put(compression.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + Channels.writeFully(channel, buffer.asReadOnlyBuffer()); } - public static CompressedFloatsIndexedSupplier fromByteBuffer( - ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper mapper - ) + private int metaSize() + { + return 1 + 4 + 4 + 1; + } + + public static CompressedFloatsIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); 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; 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 b3fd010d0ee8..000000000000 --- 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 88affd110c4e..ed916a67161f 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.io.Channels; 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 it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntIterator; import java.io.IOException; @@ -37,7 +40,6 @@ import java.nio.IntBuffer; import java.nio.channels.WritableByteChannel; import java.util.Iterator; -import java.util.List; public class CompressedIntsIndexedSupplier implements WritableSupplier { @@ -47,14 +49,14 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier> baseIntBuffers; - private final CompressedObjectStrategy.CompressionStrategy compression; + private final GenericIndexed> baseIntBuffers; + private final CompressionStrategy compression; CompressedIntsIndexedSupplier( int totalSize, int sizePer, - GenericIndexed> baseIntBuffers, - CompressedObjectStrategy.CompressionStrategy compression + GenericIndexed> baseIntBuffers, + CompressionStrategy compression ) { this.totalSize = totalSize; @@ -97,65 +99,51 @@ public int get(int index) } @Override - public long getSerializedSize() + public long getSerializedSize() throws IOException { - return 1 + // version - 4 + // totalSize - 4 + // sizePer - 1 + // compressionId - baseIntBuffers.getSerializedSize(); // data + return metaSize() + baseIntBuffers.getSerializedSize(); } @Override - public void writeToChannel(WritableByteChannel channel) throws IOException + 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()})); - baseIntBuffers.writeToChannel(channel); + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION); + meta.putInt(totalSize); + meta.putInt(sizePer); + meta.put(compression.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + baseIntBuffers.writeTo(channel, smoosher); } - public CompressedIntsIndexedSupplier convertByteOrder(ByteOrder order) + private int metaSize() { - return new CompressedIntsIndexedSupplier( - totalSize, - sizePer, - GenericIndexed.fromIterable(baseIntBuffers, CompressedIntBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)), - compression - ); + return 1 + // version + 4 + // totalSize + 4 + // sizePer + 1; // compressionId } - /** - * For testing. Do not use unless you like things breaking - */ - GenericIndexed> 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 ); } @@ -163,13 +151,13 @@ public static CompressedIntsIndexedSupplier fromByteBuffer( throw new IAE("Unknown version[%s]", versionFromBuffer); } - public static CompressedIntsIndexedSupplier fromIntBuffer(IntBuffer buffer, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression) - { - return fromIntBuffer(buffer, MAX_INTS_IN_BUFFER, byteOrder, compression); - } - - public static CompressedIntsIndexedSupplier fromIntBuffer( - final IntBuffer buffer, final int chunkFactor, final ByteOrder byteOrder, CompressedObjectStrategy.CompressionStrategy compression + @VisibleForTesting + static CompressedIntsIndexedSupplier fromIntBuffer( + final IntBuffer buffer, + final int chunkFactor, + final ByteOrder byteOrder, + final CompressionStrategy compression, + final Closer closer ) { Preconditions.checkArgument( @@ -179,15 +167,20 @@ public static CompressedIntsIndexedSupplier fromIntBuffer( return new CompressedIntsIndexedSupplier( buffer.remaining(), chunkFactor, - GenericIndexed.fromIterable( - new Iterable>() + 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() @@ -196,16 +189,17 @@ public boolean hasNext() } @Override - public ResourceHolder 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 @@ -216,17 +210,22 @@ public void remove() }; } }, - CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) + compression, + chunkFactor * Ints.BYTES, + byteOrder, + closer ), compression ); } + @VisibleForTesting public static CompressedIntsIndexedSupplier fromList( - final List list, + final IntArrayList list, final int chunkFactor, final ByteOrder byteOrder, - CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression, + final Closer closer ) { Preconditions.checkArgument( @@ -236,14 +235,18 @@ public static CompressedIntsIndexedSupplier fromList( return new CompressedIntsIndexedSupplier( list.size(), chunkFactor, - GenericIndexed.fromIterable( - new Iterable>() + 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 @@ -253,21 +256,15 @@ public boolean hasNext() } @Override - public ResourceHolder 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 @@ -278,7 +275,10 @@ public void remove() }; } }, - CompressedIntBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkFactor) + compression, + chunkFactor * Ints.BYTES, + byteOrder, + closer ), compression ); @@ -286,10 +286,10 @@ public void remove() private class CompressedIndexedInts implements IndexedInts { - final Indexed> singleThreadedIntBuffers = baseIntBuffers.singleThreaded(); + final Indexed> singleThreadedIntBuffers = baseIntBuffers.singleThreaded(); int currIndex = -1; - ResourceHolder holder; + ResourceHolder holder; IntBuffer buffer; @Override @@ -321,7 +321,7 @@ protected void loadBuffer(int bufferNum) { CloseQuietly.close(holder); holder = singleThreadedIntBuffers.get(bufferNum); - buffer = holder.get(); + buffer = holder.get().asIntBuffer(); currIndex = bufferNum; } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java index 8151dc4e27ab..1789fa4e6537 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java @@ -20,15 +20,15 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; +import io.druid.io.Channels; +import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputMedium; import io.druid.segment.IndexIO; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.IntBuffer; import java.nio.channels.WritableByteChannel; /** @@ -39,13 +39,13 @@ public class CompressedIntsIndexedWriter extends SingleValueIndexedIntsWriter private static final byte VERSION = CompressedIntsIndexedSupplier.VERSION; public static CompressedIntsIndexedWriter create( - final IOPeon ioPeon, + final OutputMedium outputMedium, final String filenameBase, - final CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression ) { return new CompressedIntsIndexedWriter( - ioPeon, + outputMedium, filenameBase, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, IndexIO.BYTE_ORDER, @@ -54,35 +54,43 @@ public static CompressedIntsIndexedWriter create( } 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 OutputMedium outputMedium, 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( + outputMedium, + chunkFactor, + byteOrder, + compression, + GenericIndexedWriter.ofCompressedByteBuffers(outputMedium, filenameBase, compression, chunkFactor * Ints.BYTES) + ); } - public CompressedIntsIndexedWriter( + CompressedIntsIndexedWriter( + final OutputMedium outputMedium, 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 = outputMedium.getCloser(); + this.endBuffer = compressor.allocateInBuffer(chunkFactor * Ints.BYTES, closer).order(byteOrder); + this.numInserted = 0; } @Override @@ -95,47 +103,50 @@ public void open() throws IOException protected void addValue(int val) throws IOException { if (!endBuffer.hasRemaining()) { - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = IntBuffer.allocate(chunkFactor); + endBuffer.clear(); + flattener.write(endBuffer); } - endBuffer.put(val); + endBuffer.putInt(val); numInserted++; } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException + { + writeEndBuffer(); + return metaSize() + flattener.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - try { - if (numInserted > 0) { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - } + writeEndBuffer(); + + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION); + meta.putInt(numInserted); + meta.putInt(chunkFactor); + meta.put(compression.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + flattener.writeTo(channel, smoosher); + } + + private void writeEndBuffer() throws IOException + { + if (endBuffer != null && numInserted > 0) { + endBuffer.flip(); + flattener.write(endBuffer); endBuffer = null; } - finally { - flattener.close(); - } } - @Override - public long getSerializedSize() + private int metaSize() { 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); + 1; // compression id } } diff --git a/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.java deleted file mode 100644 index e43e1eae12bb..000000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongBufferObjectStrategy.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.Longs; -import io.druid.java.util.common.guava.Comparators; - -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.LongBuffer; - -/** -*/ -public class CompressedLongBufferObjectStrategy extends FixedSizeCompressedObjectStrategy -{ - public static CompressedLongBufferObjectStrategy getBufferForOrder(final ByteOrder order, final CompressionStrategy compression, final int sizePer) - { - return new CompressedLongBufferObjectStrategy(order, compression, sizePer); - } - - private CompressedLongBufferObjectStrategy(final ByteOrder order, final CompressionStrategy compression, final int sizePer) - { - super( - order, - new BufferConverter() - { - @Override - public LongBuffer convert(ByteBuffer buf) - { - return buf.asLongBuffer(); - } - - @Override - public int compare(LongBuffer lhs, LongBuffer rhs) - { - return Comparators.naturalNullsFirst().compare(lhs, rhs); - } - - @Override - public int sizeOf(int count) - { - return count * Longs.BYTES; - } - - @Override - public LongBuffer combine(ByteBuffer into, LongBuffer from) - { - return into.asLongBuffer().put(from); - } - }, - compression, - sizePer - ); - } -} 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 1d701922fcbc..66fb75bf6b12 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.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.segment.serde.Serializer; import java.io.IOException; import java.nio.ByteBuffer; @@ -31,7 +33,7 @@ /** */ -public class CompressedLongsIndexedSupplier implements Supplier +public class CompressedLongsIndexedSupplier implements Supplier, Serializer { public static final byte LZF_VERSION = 0x1; public static final byte version = 0x2; @@ -41,7 +43,7 @@ public class CompressedLongsIndexedSupplier implements Supplier 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 +51,7 @@ public class CompressedLongsIndexedSupplier implements Supplier int sizePer, ByteBuffer buffer, Supplier supplier, - CompressedObjectStrategy.CompressionStrategy compression, + CompressionStrategy compression, CompressionFactory.LongEncodingFormat encoding ) { @@ -72,23 +74,34 @@ public IndexedLongs get() 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 metaSize() + (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))); + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(version); + meta.putInt(totalSize); + meta.putInt(sizePer); if (encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT) { - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); + meta.put(compression.getId()); } else { - channel.write(ByteBuffer.wrap(new byte[]{CompressionFactory.setEncodingFlag(compression.getId())})); - channel.write(ByteBuffer.wrap(new byte[]{encoding.getId()})); + meta.put(CompressionFactory.setEncodingFlag(compression.getId())); + meta.put(encoding.getId()); } - channel.write(buffer.asReadOnlyBuffer()); + meta.flip(); + + Channels.writeFully(channel, meta); + Channels.writeFully(channel, buffer.asReadOnlyBuffer()); + } + + private int metaSize() + { + return 1 + 4 + 4 + 1 + (encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT ? 0 : 1); } public static CompressedLongsIndexedSupplier fromByteBuffer( @@ -102,7 +115,7 @@ public static CompressedLongsIndexedSupplier fromByteBuffer( 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) { byte compressionId = buffer.get(); @@ -110,7 +123,7 @@ public static CompressedLongsIndexedSupplier fromByteBuffer( encoding = CompressionFactory.LongEncodingFormat.forId(buffer.get()); compressionId = CompressionFactory.clearEncodingFlag(compressionId); } - compression = CompressedObjectStrategy.CompressionStrategy.forId(compressionId); + compression = CompressionStrategy.forId(compressionId); } Supplier supplier = CompressionFactory.getLongSupplier( totalSize, @@ -118,8 +131,7 @@ public static CompressedLongsIndexedSupplier fromByteBuffer( 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 a2ee783caf1d..000000000000 --- a/processing/src/main/java/io/druid/segment/data/CompressedObjectStrategy.java +++ /dev/null @@ -1,395 +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.LZ4FastDecompressor; -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 static 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 static interface Decompressor - { - /** - * Implementations of this method are expected to call out.flip() after writing to the output buffer - * - * @param in - * @param numBytes - * @param out - */ - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out); - - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize); - } - - public static interface Compressor - { - /** - * Currently assumes buf is an array backed ByteBuffer - * - * @param bytes - * - * @return - */ - public 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 LZ4FastDecompressor lz4Fast = LZ4Factory.fastestInstance().fastDecompressor(); - 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) - { - // lz4Fast.decompress does not modify buffer positions - lz4Fast.decompress(in, in.position(), 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 static interface BufferConverter - { - public T convert(ByteBuffer buf); - - public int compare(T lhs, T rhs); - - public int sizeOf(int count); - - public 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 660127cdeb14..a97ecc8f817a 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.output.OutputMedium; 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 OutputMedium outputMedium, final String filenameBase, final int maxValue, - final CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression ) { return new CompressedVSizeIndexedV3Writer( new CompressedIntsIndexedWriter( - ioPeon, - StringUtils.format("%s.offsets", filenameBase), + outputMedium, + filenameBase, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, IndexIO.BYTE_ORDER, compression ), new CompressedVSizeIntsIndexedWriter( - ioPeon, - StringUtils.format("%s.values", filenameBase), + outputMedium, + filenameBase, maxValue, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), IndexIO.BYTE_ORDER, @@ -68,8 +68,9 @@ public static CompressedVSizeIndexedV3Writer create( 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,39 @@ public void open() throws IOException } @Override - protected void addValues(List vals) throws IOException + protected void addValues(IntList vals) throws IOException { 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 { - offsetWriter.add(offset); - } - finally { - offsetWriter.close(); - valueWriter.close(); - } + writeLastOffset(); + return 1 + offsetWriter.getSerializedSize() + valueWriter.getSerializedSize(); } @Override - public long getSerializedSize() + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - return 1 + // version - offsetWriter.getSerializedSize() + - valueWriter.getSerializedSize(); + writeLastOffset(); + Channels.writeFully(channel, ByteBuffer.wrap(new byte[]{VERSION})); + offsetWriter.writeTo(channel, smoosher); + valueWriter.writeTo(channel, smoosher); } - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private void writeLastOffset() throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{VERSION})); - offsetWriter.writeToChannel(channel, smoosher); - valueWriter.writeToChannel(channel, smoosher); + if (!lastOffsetWritten) { + offsetWriter.add(offset); + lastOffsetWritten = true; + } } } 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 c9a3c82d6f8c..d45071c0a187 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java @@ -25,13 +25,15 @@ 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.io.Channels; 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 it.unimi.dsi.fastutil.ints.IntIterator; +import it.unimi.dsi.fastutil.ints.IntList; import java.io.IOException; import java.nio.ByteBuffer; @@ -40,7 +42,6 @@ import java.nio.ShortBuffer; import java.nio.channels.WritableByteChannel; import java.util.Iterator; -import java.util.List; public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier { @@ -52,14 +53,14 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier> 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( @@ -80,10 +81,10 @@ public static int maxIntsInBufferForBytes(int numBytes) { int maxSizePer = (CompressedPools.BUFFER_SIZE - bufferPadding(numBytes)) / numBytes; // round down to the nearest power of 2 - return 1 << (Integer.SIZE - 1 - Integer.numberOfLeadingZeros(maxSizePer)); + return Integer.highestOneBit(maxSizePer); } - public static int bufferPadding(int numBytes) + static int bufferPadding(int numBytes) { // when numBytes == 3 we need to pad the buffer to allow reading an extra byte // beyond the end of the last value, since we use buffer.getInt() to read values. @@ -123,26 +124,34 @@ public IndexedInts get() } } + @Override + public long getSerializedSize() throws IOException + { + return metaSize() + baseBuffers.getSerializedSize(); + } @Override - public long getSerializedSize() + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION); + meta.put((byte) numBytes); + meta.putInt(totalSize); + meta.putInt(sizePer); + meta.put(compression.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + baseBuffers.writeTo(channel, smoosher); + } + + private int metaSize() { return 1 + // version 1 + // numBytes Ints.BYTES + // totalSize Ints.BYTES + // sizePer - 1 + // compression id - baseBuffers.getSerializedSize(); // data - } - - @Override - public void writeToChannel(WritableByteChannel channel) throws IOException - { - channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - baseBuffers.writeToChannel(channel); + 1; // compression id } @VisibleForTesting @@ -153,8 +162,7 @@ GenericIndexed> getBaseBuffers() public static CompressedVSizeIntsIndexedSupplier fromByteBuffer( ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper fileMapper + ByteOrder order ) { byte versionFromBuffer = buffer.get(); @@ -163,21 +171,14 @@ public static CompressedVSizeIntsIndexedSupplier fromByteBuffer( final int numBytes = buffer.get(); final int totalSize = buffer.getInt(); final int sizePer = buffer.getInt(); - final int chunkBytes = sizePer * numBytes + bufferPadding(numBytes); - final CompressedObjectStrategy.CompressionStrategy compression = CompressedObjectStrategy.CompressionStrategy.forId( - buffer.get() - ); + final CompressionStrategy compression = CompressionStrategy.forId(buffer.get()); return new CompressedVSizeIntsIndexedSupplier( totalSize, sizePer, numBytes, - GenericIndexed.read( - buffer, - CompressedByteBufferObjectStrategy.getBufferForOrder(order, compression, chunkBytes), - fileMapper - ), + GenericIndexed.read(buffer, new DecompressingByteBufferObjectStrategy(order, compression)), compression ); @@ -186,16 +187,18 @@ public static CompressedVSizeIntsIndexedSupplier fromByteBuffer( throw new IAE("Unknown version[%s]", versionFromBuffer); } + @VisibleForTesting public static CompressedVSizeIntsIndexedSupplier fromList( - final List 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), @@ -207,15 +210,19 @@ public static CompressedVSizeIntsIndexedSupplier fromList( list.size(), chunkFactor, numBytes, - GenericIndexed.fromIterable( - new Iterable>() + 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() @@ -224,35 +231,27 @@ public boolean hasNext() } @Override - public ResourceHolder next() + public ByteBuffer next() { - ByteBuffer retVal = ByteBuffer - .allocate(chunkBytes) - .order(byteOrder); - - if (chunkFactor > list.size() - position) { - retVal.limit((list.size() - position) * numBytes); - } else { - retVal.limit(chunkFactor * numBytes); - } + retVal.clear(); + int elementCount = Math.min(list.size() - position, chunkFactor); + retVal.limit(numBytes * elementCount); - 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 @@ -263,7 +262,10 @@ public void remove() }; } }, - CompressedByteBufferObjectStrategy.getBufferForOrder(byteOrder, compression, chunkBytes) + compression, + chunkBytes, + byteOrder, + closer ), compression ); diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java index 0df6fc3e7bdc..0645e4285426 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java @@ -20,9 +20,9 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; -import io.druid.collections.ResourceHolder; -import io.druid.collections.StupidResourceHolder; +import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputMedium; import io.druid.segment.IndexIO; import java.io.IOException; @@ -37,83 +37,78 @@ public class CompressedVSizeIntsIndexedWriter extends SingleValueIndexedIntsWrit { private static final byte VERSION = CompressedVSizeIntsIndexedSupplier.VERSION; + public static CompressedVSizeIntsIndexedWriter create( + final OutputMedium outputMedium, + final String filenameBase, + final int maxValue, + final CompressionStrategy compression + ) + { + return new CompressedVSizeIntsIndexedWriter( + outputMedium, + 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 OutputMedium outputMedium, final String filenameBase, final int maxValue, final int chunkFactor, final ByteOrder byteOrder, - final CompressedObjectStrategy.CompressionStrategy compression + final CompressionStrategy compression ) { this( - ioPeon, - filenameBase, + outputMedium, maxValue, chunkFactor, byteOrder, compression, - new GenericIndexedWriter<>( - ioPeon, + GenericIndexedWriter.ofCompressedByteBuffers( + outputMedium, filenameBase, - CompressedByteBufferObjectStrategy.getBufferForOrder( - byteOrder, - compression, - sizePer(maxValue, chunkFactor) - ) + compression, + sizePer(maxValue, chunkFactor) ) ); } - public CompressedVSizeIntsIndexedWriter( - final IOPeon ioPeon, - final String filenameBase, + CompressedVSizeIntsIndexedWriter( + final OutputMedium outputMedium, 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, outputMedium.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) @@ -130,13 +125,11 @@ public void open() throws IOException protected void addValue(int val) throws IOException { if (!endBuffer.hasRemaining()) { - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - endBuffer = ByteBuffer.allocate(chunkBytes).order(byteOrder); - endBuffer.limit(numBytes * chunkFactor); + endBuffer.clear(); + flattener.write(endBuffer); } 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); @@ -145,39 +138,44 @@ protected void addValue(int val) throws IOException } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException + { + writeEndBuffer(); + return metaSize() + flattener.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + writeEndBuffer(); + + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION); + meta.put((byte) numBytes); + meta.putInt(numInserted); + meta.putInt(chunkFactor); + meta.put(compression.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + flattener.writeTo(channel, smoosher); + } + + private void writeEndBuffer() throws IOException { - try { - if (numInserted > 0) { - endBuffer.limit(endBuffer.position()); - endBuffer.rewind(); - flattener.write(StupidResourceHolder.create(endBuffer)); - } + if (endBuffer != null && numInserted > 0) { + endBuffer.flip(); + flattener.write(endBuffer); endBuffer = null; } - finally { - flattener.close(); - } } - @Override - public long getSerializedSize() + private int metaSize() { 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); + 1; // compression id } } 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 4a0fc224c124..3f092e7a95db 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,16 @@ 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.output.OutputBytes; +import io.druid.output.OutputMedium; import java.io.IOException; -import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Map; /** - * Compression of metrics is done by using a combination of {@link 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 +215,7 @@ public static LongEncodingFormat forId(byte id) /** * This writer output encoded values to the given ByteBuffer or OutputStream. {@link #setBuffer(ByteBuffer)} or - * {@link #setOutputStream(OutputStream)} must be called before any value is written, and {@link #flush()} must + * {@link #setOutputStream(OutputBytes)} 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 +226,7 @@ public interface LongEncodingWriter */ void setBuffer(ByteBuffer buffer); - void setOutputStream(OutputStream output); + void setOutputStream(OutputBytes output); void write(long value) throws IOException; @@ -239,7 +239,9 @@ public interface LongEncodingWriter * 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 @@ -269,11 +271,10 @@ public static Supplier getLongSupplier( 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 +283,31 @@ public static Supplier getLongSupplier( fromBuffer, order, encodingFormat.getReader(fromBuffer, order), - strategy, - fileMapper + strategy ); } } public static LongSupplierSerializer getLongSerializer( - IOPeon ioPeon, String filenameBase, ByteOrder order, + OutputMedium outputMedium, + 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(outputMedium, filenameBase, order, compressionStrategy); } else if (encodingStrategy == LongEncodingStrategy.LONGS) { - if (compressionStrategy == CompressedObjectStrategy.CompressionStrategy.NONE) { - return new EntireLayoutLongSupplierSerializer( - ioPeon, filenameBase, order, new LongsLongEncodingWriter(order) - ); + if (compressionStrategy == CompressionStrategy.NONE) { + return new EntireLayoutLongSupplierSerializer(outputMedium, new LongsLongEncodingWriter(order)); } else { return new BlockLayoutLongSupplierSerializer( - ioPeon, filenameBase, order, new LongsLongEncodingWriter(order), compressionStrategy + outputMedium, + filenameBase, + order, + new LongsLongEncodingWriter(order), + compressionStrategy ); } } else { @@ -318,30 +322,27 @@ public static Supplier getFloatSupplier( 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 + OutputMedium outputMedium, + String filenameBase, + ByteOrder order, + CompressionStrategy compressionStrategy ) { - if (compressionStrategy == CompressedObjectStrategy.CompressionStrategy.NONE) { - return new EntireLayoutFloatSupplierSerializer( - ioPeon, filenameBase, order - ); + if (compressionStrategy == CompressionStrategy.NONE) { + return new EntireLayoutFloatSupplierSerializer(outputMedium, order); } else { - return new BlockLayoutFloatSupplierSerializer( - ioPeon, filenameBase, order, compressionStrategy - ); + return new BlockLayoutFloatSupplierSerializer(outputMedium, filenameBase, order, compressionStrategy); } } @@ -350,29 +351,29 @@ public static Supplier getDoubleSupplier( 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, + OutputMedium outputMedium, 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(outputMedium, byteOrder); } else { - return new BlockLayoutDoubleSupplierSerializer(ioPeon, filenameBase, byteOrder, compression); + return new BlockLayoutDoubleSupplierSerializer(outputMedium, 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 000000000000..6dbf4046d853 --- /dev/null +++ b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java @@ -0,0 +1,366 @@ +/* + * 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.java.util.common.logger.Logger; +import io.druid.segment.CompressedPools; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4FastDecompressor; +import net.jpountz.lz4.LZ4SafeDecompressor; +import org.apache.commons.lang.ArrayUtils; + +import java.io.IOException; +import java.nio.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; + + private static final Logger log = new Logger(CompressionStrategy.class); + + 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 static interface Decompressor + { + /** + * Implementations of this method are expected to call out.flip() after writing to the output buffer + * + * @param in + * @param numBytes + * @param out + */ + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out); + + public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize); + } + + 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); + } + + @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 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 LZ4FastDecompressor lz4Fast = LZ4Factory.fastestInstance().fastDecompressor(); + 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) + { + // lz4Fast.decompress does not modify buffer positions + lz4Fast.decompress(in, in.position(), out, out.position(), decompressedSize); + out.limit(out.position() + decompressedSize); + } + } + + 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 de06317428fb..44044691124c 100644 --- a/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java +++ b/processing/src/main/java/io/druid/segment/data/ConciseBitmapSerdeFactory.java @@ -19,16 +19,15 @@ package io.druid.segment.data; -import java.nio.ByteBuffer; - import com.google.common.collect.Ordering; - import io.druid.collections.bitmap.BitmapFactory; import io.druid.collections.bitmap.ConciseBitmapFactory; import io.druid.collections.bitmap.ImmutableBitmap; import io.druid.collections.bitmap.WrappedImmutableConciseBitmap; import io.druid.extendedset.intset.ImmutableConciseSet; +import java.nio.ByteBuffer; + /** */ public class ConciseBitmapSerdeFactory implements BitmapSerdeFactory @@ -68,8 +67,7 @@ public int compare( } }.nullsFirst(); - 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 000000000000..ff7a1426d17c --- /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/DeltaLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java index e73bd72f4722..03e43857fb64 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 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; +import io.druid.output.OutputBytes; import java.io.IOException; -import java.io.OutputStream; import java.nio.ByteBuffer; public class DeltaLongEncodingWriter implements CompressionFactory.LongEncodingWriter @@ -46,7 +46,7 @@ public void setBuffer(ByteBuffer buffer) } @Override - public void setOutputStream(OutputStream output) + public void setOutputStream(OutputBytes output) { serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); } @@ -58,13 +58,19 @@ public void write(long value) throws IOException } @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 ce549961dba7..edd3b6927252 100644 --- a/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/DoubleSupplierSerializer.java @@ -19,20 +19,13 @@ package io.druid.segment.data; +import io.druid.segment.serde.Serializer; -import com.google.common.io.ByteSink; -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; int size(); void add(double 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/EntireLayoutDoubleSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java index 267e30cc1741..d125e4845deb 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java @@ -19,39 +19,30 @@ 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.Doubles; import com.google.common.primitives.Ints; +import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; 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 EntireLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer { - private final IOPeon ioPeon; - private final String valueFile; - private final String metaFile; - private CountingOutputStream valuesOut; - private long metaCount = 0; - + private final OutputMedium outputMedium; private final ByteBuffer orderBuffer; + private OutputBytes valuesOut; private int numInserted = 0; - public EntireLayoutDoubleSupplierSerializer(IOPeon ioPeon, String filenameBase, ByteOrder order) + public EntireLayoutDoubleSupplierSerializer(OutputMedium outputMedium, ByteOrder order) { - this.ioPeon = ioPeon; - this.valueFile = filenameBase + ".value"; - this.metaFile = filenameBase + ".format"; + this.outputMedium = outputMedium; this.orderBuffer = ByteBuffer.allocate(Doubles.BYTES); orderBuffer.order(order); } @@ -59,7 +50,7 @@ public EntireLayoutDoubleSupplierSerializer(IOPeon ioPeon, String filenameBase, @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + valuesOut = outputMedium.makeOutputBytes(); } @Override @@ -79,46 +70,27 @@ public void add(double value) throws IOException } @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 long getSerializedSize() + public long getSerializedSize() throws IOException { - return metaCount + valuesOut.getCount(); + return metaSize() + 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); - } + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(CompressedDoublesIndexedSupplier.version); + meta.putInt(numInserted); + meta.putInt(0); + meta.put(CompressionStrategy.NONE.getId()); + meta.flip(); + + Channels.writeFully(channel, meta); + valuesOut.writeTo(channel); } - @Override - public void close() throws IOException + private int metaSize() { - 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(); - } + return 1 + Ints.BYTES + Ints.BYTES + 1; } } 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 e49c32f9f691..2b146d4cf5bb 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,35 @@ 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.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; public class EntireLayoutFloatSupplierSerializer implements FloatSupplierSerializer { - private final IOPeon ioPeon; - private final String valueFile; - private final String metaFile; - private CountingOutputStream valuesOut; - private long metaCount = 0; - - private final ByteBuffer orderBuffer; + private final boolean isLittleEndian; + private final OutputMedium outputMedium; + private OutputBytes valuesOut; private int numInserted = 0; - public EntireLayoutFloatSupplierSerializer( - IOPeon ioPeon, String filenameBase, ByteOrder order - ) + EntireLayoutFloatSupplierSerializer(OutputMedium outputMedium, ByteOrder order) { - this.ioPeon = ioPeon; - this.valueFile = filenameBase + ".value"; - this.metaFile = filenameBase + ".format"; - - orderBuffer = ByteBuffer.allocate(Floats.BYTES); - orderBuffer.order(order); + this.outputMedium = outputMedium; + isLittleEndian = order.equals(ByteOrder.LITTLE_ENDIAN); } @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + valuesOut = outputMedium.makeOutputBytes(); } @Override @@ -73,51 +59,37 @@ public int size() @Override public void add(float value) throws IOException { - orderBuffer.rewind(); - orderBuffer.putFloat(value); - valuesOut.write(orderBuffer.array()); + int valueBits = Float.floatToRawIntBits(value); + // OutputBytes 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 metaSize() + 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(); - } - } + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(CompressedFloatsIndexedSupplier.version); + meta.putInt(numInserted); + meta.putInt(0); + meta.put(CompressionStrategy.NONE.getId()); + meta.flip(); - @Override - public long getSerializedSize() - { - return metaCount + valuesOut.getCount(); + Channels.writeFully(channel, meta); + valuesOut.writeTo(channel); } - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private int metaSize() { - try (InputStream meta = ioPeon.makeInputStream(metaFile); - InputStream value = ioPeon.makeInputStream(valueFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - ByteStreams.copy(Channels.newChannel(value), channel); - } + return 1 + Ints.BYTES + Ints.BYTES + 1; } } 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 303c27df2c12..2f8d7274c093 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,34 @@ 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.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteOrder; -import java.nio.channels.Channels; +import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; public class EntireLayoutLongSupplierSerializer implements LongSupplierSerializer { - - private final IOPeon ioPeon; - private final String valueFile; - private final String metaFile; - private CountingOutputStream valuesOut; private final CompressionFactory.LongEncodingWriter writer; - private long metaCount = 0; + private final OutputMedium outputMedium; + private OutputBytes valuesOut; private int numInserted = 0; - public EntireLayoutLongSupplierSerializer( - IOPeon ioPeon, String filenameBase, ByteOrder order, - CompressionFactory.LongEncodingWriter writer - ) + EntireLayoutLongSupplierSerializer(OutputMedium outputMedium, CompressionFactory.LongEncodingWriter writer) { - this.ioPeon = ioPeon; - this.valueFile = filenameBase + ".value"; - this.metaFile = filenameBase + ".format"; + this.outputMedium = outputMedium; this.writer = writer; } @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFile)); + valuesOut = outputMedium.makeOutputBytes(); writer.setOutputStream(valuesOut); } @@ -76,45 +64,30 @@ public void add(long value) throws IOException } @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); - } + writer.flush(); + return metaSize() + valuesOut.size(); } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writer.flush(); - valuesOut.close(); - try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) { - metaOut.write(CompressedLongsIndexedSupplier.version); - metaOut.write(Ints.toByteArray(numInserted)); - metaOut.write(Ints.toByteArray(0)); - writer.putMeta(metaOut, CompressedObjectStrategy.CompressionStrategy.NONE); - metaOut.close(); - metaCount = metaOut.getCount(); - } - } - @Override - public long getSerializedSize() - { - return metaCount + valuesOut.getCount(); + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(CompressedLongsIndexedSupplier.version); + meta.putInt(numInserted); + meta.putInt(0); + writer.putMeta(meta, CompressionStrategy.NONE); + meta.flip(); + + Channels.writeFully(channel, meta); + valuesOut.writeTo(channel); } - @Override - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private int metaSize() { - try (InputStream meta = ioPeon.makeInputStream(metaFile); - InputStream value = ioPeon.makeInputStream(valueFile)) { - ByteStreams.copy(Channels.newChannel(meta), channel); - ByteStreams.copy(Channels.newChannel(value), channel); - } + return 1 + Ints.BYTES + Ints.BYTES + writer.metaSize(); } } 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 d6cf71157b62..993ab2c43ce2 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 e5a4f9a9bd33..5f5efc944864 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -20,14 +20,19 @@ package io.druid.segment.data; 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.output.HeapByteBufferOutputBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.serde.Serializer; import it.unimi.dsi.fastutil.bytes.ByteArrays; import java.io.Closeable; @@ -67,7 +72,7 @@ * 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; @@ -138,9 +143,25 @@ public static GenericIndexed fromArray(T[] objects, ObjectStrategy str 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 +329,7 @@ public Iterator iterator() return IndexedIterable.create(this).iterator(); } + @Override public long getSerializedSize() { if (!versionOne) { @@ -316,10 +338,11 @@ public long getSerializedSize() 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 +460,25 @@ private static GenericIndexed createGenericIndexedVersionOne(ByteBuffer b ); } - 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); + HeapByteBufferOutputBytes headerOut = new HeapByteBufferOutputBytes(); + HeapByteBufferOutputBytes valuesOut = new HeapByteBufferOutputBytes(); try { - int offset = 0; T prevVal = null; do { count++; @@ -462,11 +487,10 @@ private static GenericIndexed fromIterableVersionOne(Iterable objectsI 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 +506,18 @@ private static GenericIndexed fromIterableVersionOne(Iterable objectsI 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 metaSize() + theBuffer.remaining(); } private T getVersionOne(int index) @@ -518,6 +538,14 @@ private T getVersionOne(int index) return copyBufferAndGet(firstValueBuffer, startOffset, endOffset); } + private int metaSize() + { + return 1 // version byte + + 1 // allowReverseLookup flag + + Ints.BYTES // numBytesUsed + + Ints.BYTES; // numElements + } + private BufferIndexed singleThreadedVersionOne() { final ByteBuffer copyBuffer = firstValueBuffer.asReadOnlyBuffer(); @@ -552,15 +580,17 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } - 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()); + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION_ONE); + meta.put(allowReverseLookup ? REVERSE_LOOKUP_ALLOWED : REVERSE_LOOKUP_DISALLOWED); + meta.putInt(theBuffer.remaining() + 4); + meta.putInt(size); + meta.flip(); + + Channels.writeFully(channel, meta); + 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 82177a0e12af..1d51ada48e0a 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,129 @@ 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.io.Channels; 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 java.io.Closeable; -import java.io.File; -import java.io.FileInputStream; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; +import io.druid.segment.serde.Serializer; +import it.unimi.dsi.fastutil.longs.LongArrayList; +import it.unimi.dsi.fastutil.longs.LongList; + +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; + + static GenericIndexedWriter ofCompressedByteBuffers( + final OutputMedium outputMedium, + final String filenameBase, + final CompressionStrategy compressionStrategy, + final int bufferSize + ) + { + GenericIndexedWriter writer = new GenericIndexedWriter<>( + outputMedium, + filenameBase, + compressedByteBuffersWriteObjectStrategy(compressionStrategy, bufferSize, outputMedium.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, OutputBytes 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 OutputMedium outputMedium; 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 OutputBytes headerOut = null; + private OutputBytes valuesOut = null; + private int numWritten = 0; private boolean requireMultipleFiles = false; - private ByteBuffer buf; - private final ByteBuffer sizeHelperBuffer = ByteBuffer.allocate(Ints.BYTES); - + private LongList headerOutLong; - public GenericIndexedWriter( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) + public GenericIndexedWriter(OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy) { - this(ioPeon, filenameBase, strategy, Integer.MAX_VALUE & ~PAGE_SIZE); + this(outputMedium, filenameBase, strategy, Integer.MAX_VALUE & ~PAGE_SIZE); } public GenericIndexedWriter( - IOPeon ioPeon, + OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy, int fileSizeLimit ) { - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; 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 +176,8 @@ private static void writeBytesIntoSmooshedChannel( public void open() throws IOException { - headerOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("header"))); - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("values"))); + headerOut = outputMedium.makeOutputBytes(); + valuesOut = outputMedium.makeOutputBytes(); } public void write(T objectToWrite) throws IOException @@ -137,85 +186,130 @@ public void write(T objectToWrite) throws IOException objectsSorted = false; } - byte[] bytesToWrite = strategy.toBytes(objectToWrite); - ++numWritten; - SerializerUtils.writeBigEndianIntToOutputStream(valuesOut, bytesToWrite.length, sizeHelperBuffer); - valuesOut.write(bytesToWrite); + 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) + @Override + public long getSerializedSize() throws IOException { - return StringUtils.format("%s.%s", filenameBase, suffix); + if (requireMultipleFiles) { + return metaSize(); + } else { + return metaSize() + headerOut.size() + valuesOut.size(); + } } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - valuesOut.close(); if (requireMultipleFiles) { - closeMultiFiles(); + closeMultiFiles(channel, smoosher); } else { - closeSingleFile(); + writeToSingleFile(channel); } } - private void closeSingleFile() throws IOException + private void writeToSingleFile(WritableByteChannel channel) throws IOException { - headerOut.close(); - 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 < 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))); - } + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(GenericIndexed.VERSION_ONE); + meta.put(objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED); + meta.putInt(Ints.checkedCast(numBytesWritten + Integer.BYTES)); + meta.putInt(numWritten); + meta.flip(); + + Channels.writeFully(channel, meta); + headerOut.writeTo(channel); + valuesOut.writeTo(channel); } - private void closeMultiFiles() throws IOException + private void closeMultiFiles(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(); + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(GenericIndexed.VERSION_TWO); + meta.put(objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED); + meta.putInt(bagSizePower); + meta.putInt(numWritten); + meta.putInt(fileNameByteArray.length); + meta.put(fileNameByteArray); + meta.flip(); + Channels.writeFully(channel, meta); + + 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); } /** @@ -227,25 +321,19 @@ private void closeMultiFiles() throws IOException */ private int bagSizePower() throws IOException { - long avgObjectSize = (valuesOut.getCount() + numWritten - 1) / numWritten; - - File f = ioPeon.getFile(makeFilename("headerLong")); - Preconditions.checkNotNull(f, "header file missing."); + long avgObjectSize = (valuesOut.size() + numWritten - 1) / numWritten; - 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 ); } @@ -253,17 +341,16 @@ private int bagSizePower() throws IOException * 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; @@ -272,11 +359,9 @@ private boolean actuallyFits(int powerTwo, RandomAccessFile headerFile) throws I 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) { @@ -289,15 +374,13 @@ private boolean actuallyFits(int powerTwo, RandomAccessFile headerFile) throws I return true; } - public long getSerializedSize() + private int metaSize() { // 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 + Ints.BYTES; // numWritten } else { return 2 + // version and sorted flag Ints.BYTES + // numElements as log base 2. @@ -307,114 +390,17 @@ public long getSerializedSize() } } - @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++) { @@ -423,38 +409,22 @@ private void writeHeaderLong(FileSmoosher smoosher, RandomAccessFile headerFile, 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 - ); - } - - 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); - } + helperBuffer.putInt(0, Ints.checkedCast(relativeNumBytes)); + helperBuffer.clear(); + smooshChannel.write(helperBuffer); } - } } 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/IndexedIntsWriter.java b/processing/src/main/java/io/druid/segment/data/IndexedIntsWriter.java index b203b0ac7aca..4f5eac4043fa 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedIntsWriter.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedIntsWriter.java @@ -19,19 +19,13 @@ package io.druid.segment.data; -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 IndexedIntsWriter extends Closeable +public interface IndexedIntsWriter extends Serializer { public void open() throws IOException; public void add(Object obj) throws IOException; - - public long getSerializedSize(); - - public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/data/IndexedRTree.java b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java index b306e91d92f2..dd5be06b01ed 100644 --- a/processing/src/main/java/io/druid/segment/data/IndexedRTree.java +++ b/processing/src/main/java/io/druid/segment/data/IndexedRTree.java @@ -67,8 +67,7 @@ public ImmutableRTree getImmutableRTree() return immutableRTree; } - public static class ImmutableRTreeObjectStrategy - implements ObjectStrategy + public static class ImmutableRTreeObjectStrategy implements ObjectStrategy { private final BitmapFactory bitmapFactory; diff --git a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java index c775e4f3de20..48ce7b91cdb4 100644 --- a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java @@ -19,21 +19,15 @@ package io.druid.segment.data; -import com.google.common.io.ByteSink; -import com.google.common.io.CountingOutputStream; import com.google.common.math.LongMath; -import com.google.common.primitives.Longs; -import io.druid.common.utils.SerializerUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputMedium; import it.unimi.dsi.fastutil.longs.Long2IntMap; import it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap; import it.unimi.dsi.fastutil.longs.LongArrayList; import it.unimi.dsi.fastutil.longs.LongList; -import java.io.BufferedInputStream; -import java.io.DataInputStream; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; @@ -42,14 +36,11 @@ */ public class IntermediateLongSupplierSerializer implements LongSupplierSerializer { - - private final IOPeon ioPeon; + private final OutputMedium outputMedium; private final String filenameBase; - private final String tempFile; private final ByteOrder order; - private final CompressedObjectStrategy.CompressionStrategy compression; - private CountingOutputStream tempOut = null; - private final ByteBuffer helperBuffer = ByteBuffer.allocate(Longs.BYTES); + private final CompressionStrategy compression; + private LongList tempOut = null; private int numInserted = 0; @@ -61,15 +52,14 @@ public class IntermediateLongSupplierSerializer implements LongSupplierSerialize private LongSupplierSerializer delegate; - public IntermediateLongSupplierSerializer( - IOPeon ioPeon, + IntermediateLongSupplierSerializer( + OutputMedium outputMedium, String filenameBase, ByteOrder order, - CompressedObjectStrategy.CompressionStrategy compression + CompressionStrategy compression ) { - this.ioPeon = ioPeon; - this.tempFile = filenameBase + ".temp"; + this.outputMedium = outputMedium; this.filenameBase = filenameBase; this.order = order; this.compression = compression; @@ -78,7 +68,7 @@ public IntermediateLongSupplierSerializer( @Override public void open() throws IOException { - tempOut = new CountingOutputStream(ioPeon.makeOutputStream(tempFile)); + tempOut = new LongArrayList(); } @Override @@ -90,7 +80,7 @@ public int size() @Override public void add(long value) throws IOException { - SerializerUtils.writeBigEndianLongToOutputStream(tempOut, value, helperBuffer); + tempOut.add(value); ++numInserted; if (uniqueValues.size() <= CompressionFactory.MAX_TABLE_SIZE && !uniqueValues.containsKey(value)) { uniqueValues.put(value, uniqueValues.size()); @@ -106,6 +96,9 @@ public void add(long value) throws IOException private void makeDelegate() throws IOException { + if (delegate != null) { + return; + } CompressionFactory.LongEncodingWriter writer; long delta; try { @@ -122,51 +115,29 @@ private void makeDelegate() throws IOException writer = new LongsLongEncodingWriter(order); } - if (compression == CompressedObjectStrategy.CompressionStrategy.NONE) { - delegate = new EntireLayoutLongSupplierSerializer( - ioPeon, filenameBase, order, writer - ); + if (compression == CompressionStrategy.NONE) { + delegate = new EntireLayoutLongSupplierSerializer(outputMedium, writer); } else { - delegate = new BlockLayoutLongSupplierSerializer( - ioPeon, filenameBase, order, writer, compression - ); + delegate = new BlockLayoutLongSupplierSerializer(outputMedium, filenameBase, order, writer, compression); } - try (DataInputStream tempIn = new DataInputStream(new BufferedInputStream(ioPeon.makeInputStream(tempFile)))) { - delegate.open(); - int available = numInserted; - while (available > 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 - { - tempOut.close(); - makeDelegate(); - delegate.closeAndConsolidate(consolidatedOut); - } - - @Override - public void close() throws IOException + public long getSerializedSize() 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 b40554fddfe4..d03fb94760cf 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/LongsLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java index 8a32d01e999f..2a233d765a66 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.output.OutputBytes; import java.io.IOException; import java.io.OutputStream; @@ -52,7 +53,7 @@ public void setBuffer(ByteBuffer buffer) } @Override - public void setOutputStream(OutputStream output) + public void setOutputStream(OutputBytes output) { outBuffer = null; outStream = output; @@ -77,9 +78,15 @@ public void flush() throws IOException } @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 e1d096b15ce0..05d6a578045c 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 void add(Object obj) throws IOException 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 dfcb60ed4434..8ae81279b2e0 100644 --- a/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/ObjectStrategy.java @@ -20,14 +20,16 @@ package io.druid.segment.data; import io.druid.guice.annotations.ExtensionPoint; +import io.druid.output.OutputBytes; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.Comparator; @ExtensionPoint public interface ObjectStrategy extends Comparator { - public Class getClazz(); + Class getClazz(); /** * Convert values from their underlying byte representation. @@ -42,6 +44,11 @@ public interface ObjectStrategy extends Comparator * @param numBytes number of bytes used to store the value, starting at buffer.position() * @return an object created from the given byte buffer representation */ - public T fromByteBuffer(ByteBuffer buffer, int numBytes); - public byte[] toBytes(T val); + T fromByteBuffer(ByteBuffer buffer, int numBytes); + byte[] toBytes(T val); + + default void writeTo(T val, OutputBytes 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 dc7c45ad70af..0cdf4e0a2db4 100644 --- a/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java +++ b/processing/src/main/java/io/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -90,8 +90,7 @@ public int compare( } }.nullsFirst(); - 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/TableLongEncodingWriter.java b/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java index 69fde7e03fbf..ff90480fb209 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 @@ 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.output.OutputBytes; 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 void setBuffer(ByteBuffer buffer) } @Override - public void setOutputStream(OutputStream output) + public void setOutputStream(OutputBytes output) { serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); } @@ -75,18 +74,23 @@ public void flush() throws IOException } @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 c1139e61e5d2..000000000000 --- a/processing/src/main/java/io/druid/segment/data/TmpFileIOPeon.java +++ /dev/null @@ -1,100 +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(); - } - - public boolean isOverwriteAllowed() - { - return allowOverwrite; - } - - @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 1595e2b9af3a..000000000000 --- 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 16731a93bffc..f5373933811f 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java @@ -20,10 +20,11 @@ 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.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.output.HeapByteBufferOutputBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import java.io.IOException; @@ -33,7 +34,7 @@ /** */ -public class VSizeIndexed implements IndexedMultivalue +public class VSizeIndexed implements IndexedMultivalue, WritableSupplier> { private static final byte version = 0x1; @@ -41,9 +42,8 @@ public static VSizeIndexed fromIterable(Iterable objectsIterab { 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 +57,27 @@ public static VSizeIndexed fromIterable(Iterable objectsIterab ++count; } - ZeroCopyByteArrayOutputStream headerBytes = new ZeroCopyByteArrayOutputStream(4 + (count * 4)); - ZeroCopyByteArrayOutputStream valueBytes = new ZeroCopyByteArrayOutputStream(); - ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); + HeapByteBufferOutputBytes headerBytes = new HeapByteBufferOutputBytes(); + HeapByteBufferOutputBytes valueBytes = new HeapByteBufferOutputBytes(); 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,17 +147,35 @@ public int indexOf(IndexedInts value) throw new UnsupportedOperationException("Reverse lookup not allowed."); } - public int getSerializedSize() + @Override + public long getSerializedSize() throws IOException + { + return metaSize() + (long) theBuffer.remaining(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(version); + meta.put((byte) numBytes); + meta.putInt(theBuffer.remaining() + 4); + meta.putInt(size); + meta.flip(); + + Channels.writeFully(channel, meta); + Channels.writeFully(channel, theBuffer.asReadOnlyBuffer()); + } + + private int metaSize() { - return theBuffer.remaining() + 4 + 4 + 2; + return 1 + 1 + Ints.BYTES + Ints.BYTES; } - public void writeToChannel(WritableByteChannel channel) throws IOException + @Override + public IndexedMultivalue get() { - 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()); + return this; } public static VSizeIndexed readFromByteBuffer(ByteBuffer buffer) @@ -196,37 +212,4 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { 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 46d37cd72c10..168cb49c640a 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -19,20 +19,24 @@ package io.druid.segment.data; -import com.google.common.collect.Lists; 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.FileSmoosher; +import io.druid.output.HeapByteBufferOutputBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntIterator; +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.List; /** */ -public class VSizeIndexedInts implements IndexedInts, Comparable +public class VSizeIndexedInts implements IndexedInts, Comparable, WritableSupplier { public static final byte VERSION = 0x0; @@ -43,28 +47,15 @@ public static VSizeIndexedInts fromArray(int[] array) public static VSizeIndexedInts fromArray(int[] array, int maxValue) { - return fromList(Ints.asList(array), maxValue); + return fromList(IntArrayList.wrap(array), maxValue); } public static VSizeIndexedInts empty() { - return fromList(Lists.newArrayList(), 0); + return fromList(IntLists.EMPTY_LIST, 0); } - /** - * provide for performance reason. - */ - public static byte[] getBytesNoPaddingFromList(List 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); @@ -74,11 +65,11 @@ public static VSizeIndexedInts fromList(List list, int maxValue) return new VSizeIndexedInts(buffer.asReadOnlyBuffer(), numBytes); } - private static void writeToBuffer(ByteBuffer buffer, List 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); } @@ -88,7 +79,6 @@ private static void writeToBuffer(ByteBuffer buffer, List list, int num helperBuffer.putInt(0, val); buffer.put(helperBuffer.array(), Ints.BYTES - numBytes, numBytes); - ++i; } buffer.position(0); } @@ -99,15 +89,14 @@ public static byte getNumBytesForMax(int maxValue) throw new IAE("maxValue[%s] must be positive", maxValue); } - byte numBytes = 4; if (maxValue <= 0xFF) { - numBytes = 1; + return 1; } else if (maxValue <= 0xFFFF) { - numBytes = 2; + return 2; } else if (maxValue <= 0xFFFFFF) { - numBytes = 3; + return 3; } - return numBytes; + return 4; } private final ByteBuffer buffer; @@ -139,12 +128,16 @@ public int get(int index) return buffer.getInt(buffer.position() + (index * numBytes)) >>> 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(HeapByteBufferOutputBytes out) + { + ByteBuffer toWrite = buffer.slice(); + toWrite.limit(toWrite.limit() - (Ints.BYTES - numBytes)); + out.write(toWrite); } public byte[] getBytes() @@ -171,10 +164,10 @@ public int getNumBytes() return numBytes; } - public long getSerializedSize() + @Override + public long getSerializedSize() throws IOException { - // version, numBytes, size, remaining - return 1 + 1 + 4 + buffer.remaining(); + return metaSize() + buffer.remaining(); } @Override @@ -183,11 +176,29 @@ public IntIterator iterator() return new IndexedIntsIterator(this); } - public void writeToChannel(WritableByteChannel channel) throws IOException + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION); + meta.put((byte) numBytes); + meta.putInt(buffer.remaining()); + meta.flip(); + + Channels.writeFully(channel, meta); + Channels.writeFully(channel, buffer.asReadOnlyBuffer()); + } + + private int metaSize() { - channel.write(ByteBuffer.wrap(new byte[]{VERSION, (byte) numBytes})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(buffer.remaining()))); - channel.write(buffer.asReadOnlyBuffer()); + // version, numBytes, size + return 1 + 1 + Ints.BYTES; + } + + @Override + public IndexedInts get() + { + return this; } public static VSizeIndexedInts readFromByteBuffer(ByteBuffer buffer) @@ -220,37 +231,4 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("buffer", buffer); } - - public WritableSupplier 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 624aa3d4e73e..2b3c5d9cfa9d 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,14 @@ 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.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; /** @@ -38,28 +36,23 @@ public class VSizeIndexedIntsWriter extends SingleValueIndexedIntsWriter { private static final byte VERSION = VSizeIndexedInts.VERSION; - private final IOPeon ioPeon; - private final String valueFileName; + private final OutputMedium outputMedium; private final int numBytes; - private CountingOutputStream valuesOut = null; private final ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); + private OutputBytes valuesOut = null; + private boolean bufPaddingWritten = false; - public VSizeIndexedIntsWriter( - final IOPeon ioPeon, - final String filenameBase, - final int maxValue - ) + public VSizeIndexedIntsWriter(final OutputMedium outputMedium, final int maxValue) { - this.ioPeon = ioPeon; - this.valueFileName = StringUtils.format("%s.values", filenameBase); + this.outputMedium = outputMedium; this.numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue); } @Override public void open() throws IOException { - valuesOut = new CountingOutputStream(ioPeon.makeOutputStream(valueFileName)); + valuesOut = outputMedium.makeOutputBytes(); } @Override @@ -70,29 +63,40 @@ protected void addValue(int val) throws IOException } @Override - public void close() throws IOException + public long getSerializedSize() throws IOException { - byte[] bufPadding = new byte[4 - numBytes]; - valuesOut.write(bufPadding); - valuesOut.close(); + writeBufPadding(); + return metaSize() + 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(); + + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION); + meta.put((byte) numBytes); + meta.putInt(Ints.checkedCast(valuesOut.size())); + meta.flip(); + + Channels.writeFully(channel, meta); + 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; } } + + private int metaSize() + { + return 1 + // version + 1 + // numBytes + Ints.BYTES; // dataLen + } } 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 732b3722ad50..983a174abc64 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,150 @@ 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.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; +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.ByteBuffer; 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 enum WriteInt + { + ONE_BYTE { + @Override + void write(OutputBytes out, int v) throws IOException + { + out.write(v); + } + }, + TWO_BYTES { + @Override + void write(OutputBytes out, int v) throws IOException + { + out.write(v >> 8); + out.write(v); + } + }, + THREE_BYTES { + @Override + void write(OutputBytes out, int v) throws IOException + { + out.write(v >> 16); + out.write(v >> 8); + out.write(v); + } + }, + FOUR_BYTES { + @Override + void write(OutputBytes out, int v) throws IOException + { + out.writeInt(v); + } + }; + + abstract void write(OutputBytes 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; - - public VSizeIndexedWriter( - IOPeon ioPeon, - String filenameBase, - int maxId - ) + private final OutputMedium outputMedium; + private OutputBytes headerOut = null; + private OutputBytes valuesOut = null; + private int numWritten = 0; + private boolean numBytesForMaxWritten = false; + + public VSizeIndexedWriter(OutputMedium outputMedium, 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.outputMedium = outputMedium; 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 = outputMedium.makeOutputBytes(); + valuesOut = outputMedium.makeOutputBytes(); } @Override - protected void addValues(List val) throws IOException + protected void addValues(IntList ints) throws IOException { - write(val); + 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; } - public void write(List ints) throws IOException + @Override + public long getSerializedSize() throws IOException { - byte[] bytesToWrite = ints == null ? EMPTY_ARRAY : VSizeIndexedInts.getBytesNoPaddingFromList(ints, maxId); - - valuesOut.write(bytesToWrite); - - headerOut.write(Ints.toByteArray((int) valuesOut.getCount())); - - ++numWritten; + writeNumBytesForMax(); + return metaSize() + headerOut.size() + valuesOut.size(); } @Override - public void close() throws IOException + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - final byte numBytesForMax = VSizeIndexedInts.getNumBytesForMax(maxId); - - valuesOut.write(new byte[4 - numBytesForMax]); + 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 ); - try (OutputStream metaOut = ioPeon.makeOutputStream(metaFileName)) { - metaOut.write(new byte[]{VERSION, numBytesForMax}); - metaOut.write(Ints.toByteArray((int) numBytesWritten + 4)); - metaOut.write(Ints.toByteArray(numWritten)); - } + ByteBuffer meta = ByteBuffer.allocate(metaSize()); + meta.put(VERSION); + meta.put(VSizeIndexedInts.getNumBytesForMax(maxId)); + meta.putInt((int) numBytesWritten + 4); + meta.putInt(numWritten); + meta.flip(); + + Channels.writeFully(channel, meta); + 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); - } - }; - } - } - ) - ); + if (!numBytesForMaxWritten) { + final byte numBytesForMax = VSizeIndexedInts.getNumBytesForMax(maxId); + valuesOut.write(new byte[4 - numBytesForMax]); + numBytesForMaxWritten = true; + } } - @Override - public long getSerializedSize() + private int metaSize() { 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); - } + 4; // numElements } } 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 585f949724c2..4ca906b398ad 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 @@ private static final class Size1Ser implements LongSerializer ByteBuffer buffer; byte curByte = 0; int count = 0; + private boolean closed = false; public Size1Ser(OutputStream output) { @@ -222,6 +223,9 @@ public void write(long value) throws IOException @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 void close() throws IOException } else { buffer.putInt(0); } + closed = true; } } @@ -239,6 +244,7 @@ private static final class Size2Ser implements LongSerializer ByteBuffer buffer; byte curByte = 0; int count = 0; + private boolean closed = false; public Size2Ser(OutputStream output) { @@ -270,6 +276,9 @@ public void write(long value) throws IOException @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 void close() throws IOException } else { buffer.putInt(0); } + closed = true; } } @@ -289,6 +299,7 @@ private static final class Mult4Ser implements LongSerializer int numBytes; byte curByte = 0; boolean first = true; + private boolean closed = false; public Mult4Ser(OutputStream output, int numBytes) { @@ -329,6 +340,9 @@ public void write(long value) throws IOException @Override public void close() throws IOException { + if (closed) { + return; + } if (!first) { buffer.put((byte) (curByte << 4)); } @@ -339,6 +353,7 @@ public void close() throws IOException } else { buffer.putInt(0); } + closed = true; } } @@ -347,6 +362,7 @@ private static final class Mult8Ser implements LongSerializer OutputStream output; ByteBuffer buffer; int numBytes; + private boolean closed = false; public Mult8Ser(OutputStream output, int numBytes) { @@ -377,12 +393,16 @@ public void write(long value) throws IOException @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 5e704aa37ed3..49bffd57924f 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 6586d2b491f3..a49a9cc2a8d7 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 @@ 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 public Deserializer getDeserializer(); - public interface Serializer - { - public long numBytes(); - - public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; - } - public interface Deserializer { public 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 01447cb83bd3..432e1bf7a390 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 @@ 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,22 +100,7 @@ public SerializerBuilder withDelegate(final GenericColumnSerializer delegate) 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 a76878491009..91aff3fc1c7e 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 @@ 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.output.OutputMedium; 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 @@ public class ComplexColumnSerializer implements GenericColumnSerializer { - private final IOPeon ioPeon; + @PublicApi + public static ComplexColumnSerializer create(OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy) + { + return new ComplexColumnSerializer(outputMedium, filenameBase, strategy); + } + + private final OutputMedium outputMedium; private final String filenameBase; private final ObjectStrategy strategy; private GenericIndexedWriter writer; - public ComplexColumnSerializer( - IOPeon ioPeon, - String filenameBase, - ObjectStrategy strategy - ) + + private ComplexColumnSerializer(OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy) { - this.ioPeon = ioPeon; + this.outputMedium = outputMedium; 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(outputMedium, StringUtils.format("%s.complex_column", filenameBase), strategy); writer.open(); } @@ -75,28 +66,20 @@ public void serialize(Object obj) throws IOException } @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 e0b2aaa662d8..ddc03deba610 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 @@ import com.google.common.base.Function; import io.druid.guice.annotations.ExtensionPoint; +import io.druid.output.OutputMedium; 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 Object fromBytes(byte[] data, int start, int numBytes) * 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(OutputMedium outputMedium, String column) { - return ComplexColumnSerializer.create(peon, column, this.getObjectStrategy()); + return ComplexColumnSerializer.create(outputMedium, 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 cb86f47957c0..cc45a603f9d1 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -25,6 +25,7 @@ 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; @@ -211,7 +212,7 @@ public DictionaryEncodedColumnPartSerde build() new Serializer() { @Override - public long numBytes() + public long getSerializedSize() throws IOException { long size = 1 + // version (version.compareTo(VERSION.COMPRESSED) >= 0 @@ -233,23 +234,23 @@ public long numBytes() } @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, smoosher); + spatialIndexWriter.writeTo(channel, smoosher); } } } @@ -341,9 +342,9 @@ private WritableSupplier readSingleValuedColumn( { 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); } @@ -355,7 +356,7 @@ private WritableSupplier> readMultiValuedColumn( { switch (version) { case UNCOMPRESSED_MULTI_VALUE: { - return VSizeIndexed.readFromByteBuffer(buffer).asWritableSupplier(); + return VSizeIndexed.readFromByteBuffer(buffer); } case COMPRESSED: { if (Feature.MULTI_VALUE.isSet(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 f6f78e202830..5a5d8275c4f7 100644 --- a/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DoubleGenericColumnPartSerde.java @@ -19,23 +19,19 @@ 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.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 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 @@ -44,39 +40,19 @@ public static DoubleGenericColumnPartSerde getDoubleGenericColumnPartSerde( return new DoubleGenericColumnPartSerde(byteOrder, null); } - @JsonProperty - public ByteOrder getByteOrder() - { - return byteOrder; - } - + private final ByteOrder byteOrder; + private final Serializer serializer; - public DoubleGenericColumnPartSerde(ByteOrder byteOrder, Serializer serialize) + private DoubleGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) { this.byteOrder = byteOrder; - this.serialize = serialize; + this.serializer = serializer; } - @Override - public Serializer getSerializer() - { - return serialize; - } - - @Override - public Deserializer getDeserializer() + @JsonProperty + public ByteOrder getByteOrder() { - return (buffer, builder, columnConfig) -> { - final CompressedDoublesIndexedSupplier column = CompressedDoublesIndexedSupplier.fromByteBuffer( - buffer, - byteOrder, - builder.getFileMapper() - ); - builder.setType(ValueType.DOUBLE) - .setHasMultipleValues(false) - .setGenericColumn(new DoubleGenericColumnSupplier(column)); - - }; + return byteOrder; } public static SerializerBuilder serializerBuilder() @@ -89,8 +65,7 @@ public static class SerializerBuilder private ByteOrder byteOrder = null; private DoubleColumnSerializer delegate = null; - public - SerializerBuilder withByteOrder(final ByteOrder byteOrder) + public SerializerBuilder withByteOrder(final ByteOrder byteOrder) { this.byteOrder = byteOrder; return this; @@ -104,23 +79,33 @@ public SerializerBuilder withDelegate(final DoubleColumnSerializer delegate) 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 CompressedDoublesIndexedSupplier 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 fe2ca9705fe4..bd26cfb9a005 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 @@ 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 static FloatGenericColumnPartSerde createDeserializer( } private final ByteOrder byteOrder; - private Serializer serializer; + private final Serializer serializer; private FloatGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) { @@ -85,22 +81,7 @@ public SerializerBuilder withDelegate(final FloatColumnSerializer delegate) 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); } } @@ -120,12 +101,11 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo { final CompressedFloatsIndexedSupplier column = CompressedFloatsIndexedSupplier.fromByteBuffer( buffer, - byteOrder, - builder.getFileMapper() + byteOrder ); builder.setType(ValueType.FLOAT) .setHasMultipleValues(false) - .setGenericColumn(new FloatGenericColumnSupplier(column, byteOrder)); + .setGenericColumn(new FloatGenericColumnSupplier(column)); } }; } diff --git a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java index 81c9a91e4672..0e81aa82a097 100644 --- a/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java +++ b/processing/src/main/java/io/druid/segment/serde/FloatGenericColumnSupplier.java @@ -24,19 +24,15 @@ import io.druid.segment.column.IndexedFloatsGenericColumn; import io.druid.segment.data.CompressedFloatsIndexedSupplier; -import java.nio.ByteOrder; - /** */ public class FloatGenericColumnSupplier implements Supplier { private final CompressedFloatsIndexedSupplier column; - private final ByteOrder byteOrder; - public FloatGenericColumnSupplier(CompressedFloatsIndexedSupplier column, ByteOrder byteOrder) + public FloatGenericColumnSupplier(CompressedFloatsIndexedSupplier column) { this.column = column; - this.byteOrder = byteOrder; } @Override 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 f3c12d8a919f..e8e762389da0 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 @@ 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.output.OutputMedium; 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,51 +32,52 @@ public class LargeColumnSupportedComplexColumnSerializer implements GenericColumnSerializer { - - private final IOPeon ioPeon; - private final String filenameBase; - private final ObjectStrategy strategy; - private final int columnSize; - private GenericIndexedWriter writer; - public LargeColumnSupportedComplexColumnSerializer( - IOPeon ioPeon, + @PublicApi + public static LargeColumnSupportedComplexColumnSerializer create( + OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy ) { - this(ioPeon, filenameBase, strategy, Integer.MAX_VALUE); + return new LargeColumnSupportedComplexColumnSerializer(outputMedium, filenameBase, strategy); } - public LargeColumnSupportedComplexColumnSerializer( - IOPeon ioPeon, + + public static LargeColumnSupportedComplexColumnSerializer createWithColumnSize( + OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy, int columnSize ) { - this.ioPeon = ioPeon; - this.filenameBase = filenameBase; - this.strategy = strategy; - this.columnSize = columnSize; + return new LargeColumnSupportedComplexColumnSerializer(outputMedium, filenameBase, strategy, columnSize); } - @PublicApi - public static LargeColumnSupportedComplexColumnSerializer create( - IOPeon ioPeon, + private final OutputMedium outputMedium; + private final String filenameBase; + private final ObjectStrategy strategy; + private final int columnSize; + private GenericIndexedWriter writer; + + private LargeColumnSupportedComplexColumnSerializer( + OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy ) { - return new LargeColumnSupportedComplexColumnSerializer(ioPeon, filenameBase, strategy); + this(outputMedium, filenameBase, strategy, Integer.MAX_VALUE); } - public static LargeColumnSupportedComplexColumnSerializer createWithColumnSize( - IOPeon ioPeon, + private LargeColumnSupportedComplexColumnSerializer( + OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy, int columnSize ) { - return new LargeColumnSupportedComplexColumnSerializer(ioPeon, filenameBase, strategy, columnSize); + this.outputMedium = outputMedium; + this.filenameBase = filenameBase; + this.strategy = strategy; + this.columnSize = columnSize; } @SuppressWarnings(value = "unchecked") @@ -84,7 +85,11 @@ public static LargeColumnSupportedComplexColumnSerializer createWithColumnSize( public void open() throws IOException { writer = new GenericIndexedWriter( - ioPeon, StringUtils.format("%s.complex_column", filenameBase), strategy, columnSize); + outputMedium, + StringUtils.format("%s.complex_column", filenameBase), + strategy, + columnSize + ); writer.open(); } @@ -96,21 +101,15 @@ public void serialize(Object obj) throws IOException } @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 e5f943be0bb5..e33ef1c974f2 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 @@ 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 static LongGenericColumnPartSerde createDeserializer( } private final ByteOrder byteOrder; - private Serializer serializer; + private final Serializer serializer; private LongGenericColumnPartSerde(ByteOrder byteOrder, Serializer serializer) { @@ -85,22 +81,7 @@ public SerializerBuilder withDelegate(final LongColumnSerializer delegate) 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); } } 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 000000000000..e59da0e971ec --- /dev/null +++ b/processing/src/main/java/io/druid/segment/serde/Serializer.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.serde; + +import io.druid.java.util.common.io.smoosh.FileSmoosher; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public interface Serializer +{ + long getSerializedSize() throws IOException; + void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException; +} diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index fb0b18a80c22..5c10a9d79558 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.granularity.Granularities; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; import io.druid.query.aggregation.AggregationTestHelper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; @@ -62,8 +65,8 @@ 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 @@ @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, TmpFileOutputMediumFactory.instance()}); + constructors.add(new Object[]{config, OffHeapMemoryOutputMediumFactory.instance()}); + } + return constructors; + } + + private final AggregationTestHelper helper; + private final OutputMediumFactory outputMediumFactory; - private static IncrementalIndex incrementalIndex; - private static QueryableIndex queryableIndex; + private IncrementalIndex incrementalIndex; + private QueryableIndex queryableIndex; - private static File persistedSegmentDir; + private File persistedSegmentDir; - public MultiValuedDimensionTest( - final GroupByQueryConfig config - ) throws Exception + public MultiValuedDimensionTest(final GroupByQueryConfig config, OutputMediumFactory outputMediumFactory) + throws Exception { helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( ImmutableList.of(), config, null ); + this.outputMediumFactory = outputMediumFactory; } - @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 static void setupClass() throws Exception } persistedSegmentDir = Files.createTempDir(); - TestHelper.getTestIndexMergerV9() - .persist(incrementalIndex, persistedSegmentDir, new IndexSpec()); + TestHelper.getTestIndexMergerV9(outputMediumFactory) + .persist(incrementalIndex, persistedSegmentDir, new IndexSpec(), null); - queryableIndex = TestHelper.getTestIndexIO().loadIndex(persistedSegmentDir); + queryableIndex = TestHelper.getTestIndexIO(outputMediumFactory).loadIndex(persistedSegmentDir); } @Test @@ -331,8 +336,8 @@ public void testTopNWithDimFilterAndWithFilteredDimSpec() throws Exception ); } - @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 0c4d9b620018..e56c7fa07bae 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -47,6 +47,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -143,6 +144,7 @@ public static final AggregationTestHelper createGroupByQueryAggregationTestHelpe IndexIO indexIO = new IndexIO( mapper, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -155,7 +157,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), indexIO, factory.getToolchest(), factory, @@ -199,6 +201,7 @@ public static final AggregationTestHelper createSelectQueryAggregationTestHelper IndexIO indexIO = new IndexIO( mapper, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -211,7 +214,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), indexIO, toolchest, factory, @@ -239,6 +242,7 @@ public static final AggregationTestHelper createTimeseriesQueryAggregationTestHe IndexIO indexIO = new IndexIO( mapper, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -251,7 +255,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), indexIO, toolchest, factory, @@ -290,6 +294,7 @@ public ByteBuffer get() IndexIO indexIO = new IndexIO( mapper, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -302,7 +307,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO), + new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), indexIO, toolchest, factory, @@ -431,7 +436,7 @@ public void createIndex( 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( @@ -457,19 +462,19 @@ public void createIndex( 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 d9d4382ee821..81c909365b15 100644 --- a/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/io/druid/query/aggregation/FilteredAggregatorTest.java @@ -51,8 +51,8 @@ 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; @@ -108,9 +108,9 @@ public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) public IndexedInts getRow() { if (selector.getIndex() % 3 == 2) { - return ArrayBasedIndexedInts.of(new int[]{1}); + return new SingleIndexedInt(1); } else { - return ArrayBasedIndexedInts.of(new int[]{0}); + return new SingleIndexedInt(0); } } 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 1f3bcdf7946d..6f6ad24e5533 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.math.expr.ExprMacroTable; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -122,7 +124,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } @@ -172,7 +174,8 @@ public void setup() throws Exception 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 void setup() throws Exception 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 73ff071087b9..8ab59374631a 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.Pair; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.Druids; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; @@ -112,10 +113,11 @@ public class AppendTest @Before public void setUp() throws Exception { + SchemalessIndexTest schemalessIndexTest = new SchemalessIndexTest(OffHeapMemoryOutputMediumFactory.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) @@ -130,7 +132,7 @@ public void setUp() throws Exception // (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) @@ -146,7 +148,7 @@ public void setUp() throws Exception // |-------------| // |---| // |---| - 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 e807bf165e6f..2502730e0946 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 @@ 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 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 void setUpSimple() 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 IndexedInts apply(int[] input) } } ), 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; } diff --git a/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java b/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java index 595a1e80e046..c9e15a5f834f 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.output.OutputMediumFactory; 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, + OutputMediumFactory outputMediumFactory ) { - super(new ConciseBitmapSerdeFactory(), compressionStrategy, dimCompressionStrategy, longEncodingStrategy); - indexMerger = TestHelper.getTestIndexMergerV9(); + super( + new ConciseBitmapSerdeFactory(), + compressionStrategy, + dimCompressionStrategy, + longEncodingStrategy, + outputMediumFactory + ); + indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); } } diff --git a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java index 99654c6ff188..28fed060ee7f 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.Column; import io.druid.segment.incremental.IncrementalIndex; @@ -30,11 +34,33 @@ 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[] {TmpFileOutputMediumFactory.instance()}, + new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + ); + } + + private final OutputMediumFactory outputMediumFactory; + + public EmptyIndexTest(OutputMediumFactory outputMediumFactory) + { + this.outputMediumFactory = outputMediumFactory; + } + @Test public void testEmptyIndex() throws Exception { @@ -57,7 +83,7 @@ public void testEmptyIndex() throws Exception emptyIndex, new ConciseBitmapFactory() ); - TestHelper.getTestIndexMergerV9().merge( + TestHelper.getTestIndexMergerV9(outputMediumFactory).merge( Lists.newArrayList(emptyIndexAdapter), true, new AggregatorFactory[0], @@ -65,7 +91,7 @@ public void testEmptyIndex() throws Exception new IndexSpec() ); - QueryableIndex emptyQueryableIndex = TestHelper.getTestIndexIO().loadIndex(tmpDir); + QueryableIndex emptyQueryableIndex = TestHelper.getTestIndexIO(outputMediumFactory).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 7e3e65257d5a..820c89f2147b 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.Lists; import io.druid.data.input.InputRow; import io.druid.java.util.common.StringUtils; +import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; 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 OutputMediumFactory outputMediumFactory = OffHeapMemoryOutputMediumFactory.instance(); + private IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); private File tmpDir; private IndexSpec indexSpec = new IndexSpec(); private int maxRows = DEFAULT_MAX_ROWS; @@ -73,9 +76,10 @@ public IndexBuilder schema(IncrementalIndexSchema schema) return this; } - public IndexBuilder indexMerger(IndexMerger indexMerger) + public IndexBuilder outputMediumFactory(OutputMediumFactory outputMediumFactory) { - this.indexMerger = indexMerger; + this.outputMediumFactory = outputMediumFactory; + this.indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); return this; } @@ -119,11 +123,12 @@ public QueryableIndex buildMMappedIndex() Preconditions.checkNotNull(indexMerger, "indexMerger"); Preconditions.checkNotNull(tmpDir, "tmpDir"); try (final IncrementalIndex incrementalIndex = buildIncrementalIndex()) { - return TestHelper.getTestIndexIO().loadIndex( + return TestHelper.getTestIndexIO(outputMediumFactory).loadIndex( indexMerger.persist( incrementalIndex, new File(tmpDir, StringUtils.format("testIndex-%s", new Random().nextInt(Integer.MAX_VALUE))), - indexSpec + indexSpec, + null ) ); } @@ -134,14 +139,14 @@ public QueryableIndex buildMMappedIndex() public QueryableIndex buildMMappedMergedIndex() { - Preconditions.checkNotNull(indexMerger, "indexMerger"); + IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); 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(outputMediumFactory).loadIndex( indexMerger.persist( buildIncrementalIndexWithRows( schema, @@ -149,12 +154,13 @@ public QueryableIndex buildMMappedMergedIndex() 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(outputMediumFactory).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 d4470ee9dcf6..ced720c2e975 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.impl.DimensionsSpec; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.UOE; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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 ); @@ -329,7 +330,7 @@ public void testRowValidatorEquals() throws Exception { Exception ex = null; try { - TestHelper.getTestIndexIO().validateTwoSegments(adapter1, adapter2); + TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()).validateTwoSegments(adapter1, adapter2); } catch (Exception e) { ex = e; diff --git a/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java index f5196d062a52..4861c222b225 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.ISE; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import io.druid.output.OutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -49,8 +50,8 @@ import io.druid.segment.column.DictionaryEncodedColumn; import io.druid.segment.column.SimpleDictionaryEncodedColumn; import io.druid.segment.data.BitmapSerdeFactory; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.data.IndexedInts; import io.druid.segment.incremental.IncrementalIndex; @@ -80,22 +81,22 @@ 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}, output 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), + OutputMediumFactory.builtInFactories() ) ), new Function, Object[]>() { @@ -111,8 +112,8 @@ public Object[] apply(List input) static IndexSpec makeIndexSpec( BitmapSerdeFactory bitmapSerdeFactory, - CompressedObjectStrategy.CompressionStrategy compressionStrategy, - CompressedObjectStrategy.CompressionStrategy dimCompressionStrategy, + CompressionStrategy compressionStrategy, + CompressionStrategy dimCompressionStrategy, CompressionFactory.LongEncodingStrategy longEncodingStrategy ) { @@ -129,17 +130,26 @@ static IndexSpec makeIndexSpec( } 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, + OutputMediumFactory outputMediumFactory ) { - this.indexSpec = makeIndexSpec(bitmapSerdeFactory, compressionStrategy, dimCompressionStrategy, longEncodingStrategy); + this.indexSpec = makeIndexSpec( + bitmapSerdeFactory, + compressionStrategy, + dimCompressionStrategy, + longEncodingStrategy + ); + this.indexIO = TestHelper.getTestIndexIO(outputMediumFactory); } @Test @@ -152,11 +162,12 @@ public void testPersist() throws Exception final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -199,11 +210,12 @@ public void testPersistWithDifferentDims() throws Exception final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -241,11 +253,12 @@ public void testPersistWithSegmentMetadata() throws Exception final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -301,11 +314,12 @@ public void testPersistMerge() throws Exception final File mergedDir = temporaryFolder.newFolder(); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -315,11 +329,12 @@ public void testPersistMerge() throws Exception Assert.assertEquals(3, index1.getColumnNames().size()); QueryableIndex index2 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist2, tempDir2, - indexSpec + indexSpec, + null ) ) ); @@ -332,13 +347,14 @@ public void testPersistMerge() throws Exception new CountAggregatorFactory("count") }; QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( Arrays.asList(index1, index2), true, mergedAggregators, mergedDir, - indexSpec + indexSpec, + null ) ) ); @@ -390,31 +406,34 @@ public void testPersistEmptyColumn() throws Exception ); 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 ) ) ); @@ -450,11 +469,12 @@ public void testMergeRetainsValues() throws Exception ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -462,7 +482,7 @@ public void testMergeRetainsValues() throws Exception 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())); @@ -470,13 +490,14 @@ public void testMergeRetainsValues() throws Exception QueryableIndex merged = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.mergeQueryableIndex( ImmutableList.of(index1), true, new AggregatorFactory[]{new CountAggregatorFactory("count")}, mergedDir, - indexSpec + indexSpec, + null ) ) ); @@ -485,7 +506,7 @@ public void testMergeRetainsValues() throws Exception 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()); @@ -508,15 +529,11 @@ public void testAppendRetainsValues() throws Exception ); 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())); @@ -529,13 +546,14 @@ public void testAppendRetainsValues() throws Exception 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 ) ) ); @@ -544,7 +562,7 @@ public void testAppendRetainsValues() throws Exception 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()); @@ -572,11 +590,12 @@ public void testMergeSpecChange() throws Exception ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -584,7 +603,7 @@ public void testMergeSpecChange() throws Exception 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())); @@ -592,12 +611,12 @@ public void testMergeSpecChange() throws Exception 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 @@ -605,13 +624,14 @@ public void testMergeSpecChange() throws Exception 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 ) ) ); @@ -620,7 +640,7 @@ public void testMergeSpecChange() throws Exception 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()); @@ -652,12 +672,12 @@ public void testConvertSame() throws Exception ); 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())); @@ -665,7 +685,7 @@ public void testConvertSame() throws Exception QueryableIndex converted = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.convert( tempDir1, convertDir, @@ -678,7 +698,7 @@ public void testConvertSame() throws Exception 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()); @@ -715,11 +735,12 @@ public void testConvertDifferent() throws Exception ); QueryableIndex index1 = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( toPersist1, tempDir1, - indexSpec + indexSpec, + null ) ) ); @@ -727,7 +748,7 @@ public void testConvertDifferent() throws Exception 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())); @@ -736,19 +757,19 @@ public void testConvertDifferent() throws Exception 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, @@ -761,7 +782,7 @@ public void testConvertDifferent() throws Exception 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()); @@ -772,11 +793,11 @@ public void testConvertDifferent() throws Exception ); } - 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; } @@ -822,44 +843,48 @@ public void testNonLexicographicDimOrderMerge() throws Exception 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 ) ) ); @@ -930,43 +955,47 @@ public void testMergeWithDimensionsList() throws Exception 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 ) ) ); @@ -1011,55 +1040,60 @@ public void testDisjointDimMerge() throws Exception 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 ) ) ); @@ -1179,33 +1213,36 @@ public void testJointDimMerge() throws Exception 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 ) ) ); @@ -1321,33 +1358,36 @@ public void testNoRollupMergeWithoutDuplicateRow() throws Exception 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 ) ) ); @@ -1462,33 +1502,36 @@ public void testNoRollupMergeWithDuplicateRow() throws Exception 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 ) ) ); @@ -1572,75 +1615,82 @@ public void testMergeWithSupersetOrdering() throws Exception 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 ) ) ); @@ -1817,7 +1867,7 @@ public void testAddMetrics() throws IOException 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())); @@ -1888,7 +1938,7 @@ public void testAddMetricsBothSidesNull() throws IOException 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())); @@ -1953,7 +2003,7 @@ public void testMismatchedMetrics() throws IOException ); // 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())); @@ -1995,7 +2045,7 @@ public void testMismatchedMetricsVarying() throws IOException 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())); } @@ -2011,33 +2061,36 @@ public void testMergeNumericDims() throws Exception 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 ) ) ); @@ -2142,11 +2195,12 @@ public void testPersistNullColumnSkipping() throws Exception final File tempDir = temporaryFolder.newFolder(); QueryableIndex index = closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( indexMerger.persist( index1, tempDir, - indexSpec + indexSpec, + null ) ) ); @@ -2287,7 +2341,8 @@ public void testCloser() throws Exception indexMerger.persist( toPersist, tempDir, - indexSpec + indexSpec, + null ); } finally { @@ -2406,6 +2461,6 @@ private QueryableIndex persistAndLoad(List schema, InputRow... } 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 f116a2c022a8..02a360ce2e81 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -23,13 +23,16 @@ import com.google.common.collect.ImmutableMap; import com.google.common.io.ByteSource; import com.google.common.io.Files; -import io.druid.java.util.common.JodaUtils; import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; +import io.druid.java.util.common.JodaUtils; +import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; 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.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.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[] {TmpFileOutputMediumFactory.instance()}, + new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + ); + } + private static final long TIMESTAMP = DateTime.parse("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(OutputMediumFactory outputMediumFactory) { + indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + indexIO = TestHelper.getTestIndexIO(outputMediumFactory); events = new ArrayList<>(); final Map map1 = ImmutableMap.of( @@ -161,7 +177,7 @@ public void testPersistWithSegmentMetadata() throws IOException 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 void testSimpleReprocess() throws IOException { final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( persistTmpDir ) ) @@ -192,12 +208,12 @@ public void testSimpleReprocess() throws IOException 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 void testIdempotentReprocess() throws IOException { final IndexableAdapter adapter = new QueryableIndexIndexableAdapter( closer.closeLater( - INDEX_IO.loadIndex( + indexIO.loadIndex( persistTmpDir ) ) @@ -216,12 +232,12 @@ public void testIdempotentReprocess() throws IOException 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 9c17dc9078dc..8a36102546c0 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.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.output.OutputMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -56,6 +57,7 @@ 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 @@ @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 (OutputMediumFactory outputMediumFactory : OutputMediumFactory.builtInFactories()) { + IndexMergerV9 indexMergerV9 = TestHelper.getTestIndexMergerV9(outputMediumFactory); + IndexIO indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + + 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 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException 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 @@ private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOE 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 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) 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 68223b5680c2..3fbd76e0c152 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 @@ 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 void testSerde() throws Exception 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 void testSerdeUncompressed() throws Exception 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 void testSerdeUncompressed() throws Exception 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 a08cb561cd61..4ae14088e94f 100644 --- a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java @@ -19,8 +19,12 @@ package io.druid.segment; -import io.druid.segment.data.CompressedObjectStrategy; +import com.google.common.collect.ImmutableList; +import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; 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.data.IndexedInts; @@ -29,25 +33,47 @@ 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[] {TmpFileOutputMediumFactory.instance()}, + new Object[] {OffHeapMemoryOutputMediumFactory.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(OutputMediumFactory outputMediumFactory) + { + indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + } + @Test public void testGetBitmapIndex() throws Exception { @@ -57,18 +83,19 @@ public void testGetBitmapIndex() throws Exception 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 IndexedInts indexedInts = adapter.getBitmapIndex(dimension, 0); for (int i = 0; i < adapter.getDimValueLookup(dimension).size(); i++) { indexedInts = adapter.getBitmapIndex(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 c5e59470be9e..d149eede7272 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.output.OutputMediumFactory; 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, + OutputMediumFactory outputMediumFactory ) { - super(new RoaringBitmapSerdeFactory(null), compressionStrategy, dimCompressionStrategy, longEncodingStrategy); - indexMerger = TestHelper.getTestIndexMergerV9(); + super( + new RoaringBitmapSerdeFactory(null), + compressionStrategy, + dimCompressionStrategy, + longEncodingStrategy, + outputMediumFactory + ); + indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); } } diff --git a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java index ca2caca63f29..5bafa9493d39 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.granularity.Granularities; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OutputMediumFactory; 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(OutputMediumFactory outputMediumFactory) + { + indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + } + public static IncrementalIndex getIncrementalIndex() { synchronized (log) { @@ -178,7 +185,7 @@ public static QueryableIndex getIncrementalIndex(int index1, int index2) } } - public static QueryableIndex getMergedIncrementalIndex() + public QueryableIndex getMergedIncrementalIndex() { synchronized (log) { if (mergedIndex != null) { @@ -203,16 +210,17 @@ public static QueryableIndex getMergedIncrementalIndex() 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 static QueryableIndex getMergedIncrementalIndex() } } - 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 static QueryableIndex getMergedIncrementalIndex(int index1, int index2) 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 static QueryableIndex getMergedIncrementalIndex(int index1, int index2) } } - public static QueryableIndex getMergedIncrementalIndex(int[] indexes) + public QueryableIndex getMergedIncrementalIndex(int[] indexes) { synchronized (log) { if (rowPersistedIndexes.isEmpty()) { @@ -293,8 +302,8 @@ public static QueryableIndex getMergedIncrementalIndex(int[] indexes) 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 static QueryableIndex getMergedIncrementalIndex(int[] indexes) } } - public static QueryableIndex getAppendedIncrementalIndex( + public QueryableIndex getAppendedIncrementalIndex( Iterable> files, List intervals ) @@ -311,7 +320,7 @@ public static QueryableIndex getAppendedIncrementalIndex( return makeAppendedMMappedIndex(files, intervals); } - public static QueryableIndex getMergedIncrementalIndexDiffMetrics() + public QueryableIndex getMergedIncrementalIndexDiffMetrics() { return getMergedIncrementalIndex( Arrays.>asList( @@ -321,7 +330,7 @@ public static QueryableIndex getMergedIncrementalIndexDiffMetrics() ); } - public static QueryableIndex getMergedIncrementalIndex(Iterable> files) + public QueryableIndex getMergedIncrementalIndex(Iterable> files) { return makeMergedMMappedIndex(files); } @@ -342,7 +351,7 @@ private static void makeEvents() } } - private static void makeRowPersistedIndexes() + private void makeRowPersistedIndexes() { synchronized (log) { try { @@ -380,8 +389,8 @@ private static void makeRowPersistedIndexes() 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 static IncrementalIndex makeIncrementalIndex(final String resourceFilenam 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 @@ private static List makeFilesToMap(File tmpFile, Iterable> files, final List intervals ) @@ -498,7 +506,7 @@ public IndexableAdapter apply(PartitionChunk chunk) { 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 boolean apply(Rowboat input) ) ); - 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 @@ private static QueryableIndex makeMergedMMappedIndex(Iterable 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 @@ private static QueryableIndex makeMergedMMappedIndex(Iterable constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileOutputMediumFactory.instance()}, + new Object[] {OffHeapMemoryOutputMediumFactory.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 dimensionValue = "dimension"; @@ -97,6 +116,11 @@ public class SchemalessTestFullTest Arrays.asList(Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) ); + public SchemalessTestFullTest(OutputMediumFactory outputMediumFactory) + { + schemalessIndexTest = new SchemalessIndexTest(outputMediumFactory); + } + @Test public void testCompleteIntersectingSchemas() { @@ -929,7 +953,7 @@ public void testEmptySchemas() runTests( new QueryableIndexSegment( - null, SchemalessIndexTest.getMergedIncrementalIndex(0, 0) + null, schemalessIndexTest.getMergedIncrementalIndex(0, 0) ), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, @@ -1014,7 +1038,7 @@ public void testExactSameSchemas() runTests( new QueryableIndexSegment( - null, SchemalessIndexTest.getMergedIncrementalIndex(1, 1) + null, schemalessIndexTest.getMergedIncrementalIndex(1, 1) ), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, @@ -1145,7 +1169,7 @@ public void testMultiDimensionalValues() ); runTests( - new QueryableIndexSegment(null, SchemalessIndexTest.getMergedIncrementalIndex(new int[]{6, 7, 8})), + new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndex(new int[]{6, 7, 8})), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, expectedTopNResults, @@ -1336,7 +1360,7 @@ public void testDifferentMetrics() ); runTests( - new QueryableIndexSegment(null, SchemalessIndexTest.getMergedIncrementalIndexDiffMetrics()), + new QueryableIndexSegment(null, schemalessIndexTest.getMergedIncrementalIndexDiffMetrics()), expectedTimeseriesResults, expectedFilteredTimeSeriesResults, expectedTopNResults, @@ -1361,11 +1385,11 @@ private List> getIndexes(int index1, int index2) 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 69facab0d421..250ed797c75a 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.Intervals; import io.druid.java.util.common.granularity.Granularities; import io.druid.java.util.common.granularity.Granularity; +import io.druid.output.OutputMediumFactory; import io.druid.query.Druids; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; @@ -60,6 +61,7 @@ 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 (OutputMediumFactory outputMediumFactory : OutputMediumFactory.builtInFactories()) { + SchemalessIndexTest schemalessIndexTest = new SchemalessIndexTest(outputMediumFactory); + 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 6103abfd8678..acc9abd57937 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.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 a0f5e62d5d21..b37b4d4887dd 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.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.math.expr.ExprMacroTable; +import io.druid.output.OutputMediumFactory; import io.druid.query.Result; import io.druid.query.expression.TestExprMacroTable; import io.druid.query.timeseries.TimeseriesResultValue; @@ -46,13 +47,18 @@ */ 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(OutputMediumFactory outputMediumFactory) + { + return new IndexMergerV9(JSON_MAPPER, getTestIndexIO(outputMediumFactory), outputMediumFactory); + } + + public static IndexIO getTestIndexIO(OutputMediumFactory outputMediumFactory) + { + return new IndexIO( + JSON_MAPPER, + outputMediumFactory, new ColumnConfig() { @Override @@ -62,17 +68,6 @@ public int columnCacheSizeBytes() } } ); - 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 4271eaa213b4..edd181f90efb 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.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.logger.Logger; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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(OffHeapMemoryOutputMediumFactory.instance()); + private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()); static { if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { @@ -225,8 +227,8 @@ public static QueryableIndex mergedRealtimeIndex() 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 static QueryableIndex mergedRealtimeIndex() true, METRIC_AGGS, mergedFile, - indexSpec + indexSpec, + null ) ); @@ -362,7 +365,7 @@ public static QueryableIndex persistRealtimeAndLoadMMapped(IncrementalIndex inde 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 16c58c01a248..c5833e34dc49 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.output.OffHeapMemoryOutputMedium; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.Assert; import org.junit.Test; @@ -32,7 +32,6 @@ 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 static Iterable compressionStrategies() 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 static Iterable compressionStrategies() }; public CompressedFloatsSerdeTest( - CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressionStrategy compressionStrategy, ByteOrder order ) { @@ -108,7 +107,11 @@ public void testChunkSerde() throws Exception public void testWithValues(float[] values) throws Exception { - FloatSupplierSerializer serializer = CompressionFactory.getFloatSerializer(new IOPeonForTesting(), "test", order, compressionStrategy + FloatSupplierSerializer serializer = CompressionFactory.getFloatSerializer( + new OffHeapMemoryOutputMedium(), + "test", + order, + compressionStrategy ); serializer.open(); @@ -118,19 +121,10 @@ public void testWithValues(float[] values) throws Exception 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 @@ private void assertIndexMatchesVals(IndexedFloats indexed, float[] vals) 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 455b1af0d31f..adc3f8c40c31 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.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 @@ 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 CompressedIntsIndexedSupplierTest(CompressedObjectStrategy.CompressionStr @Before public void setUp() throws Exception { + closer = Closer.create(); CloseQuietly.close(indexed); indexed = null; supplier = null; @@ -65,6 +68,7 @@ public void setUp() throws Exception @After public void tearDown() throws Exception { + closer.close(); CloseQuietly.close(indexed); } @@ -78,7 +82,8 @@ private void setupSimple(final int chunkSize) IntBuffer.wrap(vals), chunkSize, ByteOrder.nativeOrder(), - compressionStrategy + compressionStrategy, + closer ); indexed = supplier.get(); @@ -97,14 +102,14 @@ private void makeWithSerde(final int chunkSize) throws IOException 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 303f10c8b9cf..a0fd7e7fb701 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 @@ 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.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.output.OffHeapMemoryOutputMedium; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; +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.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 OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); + 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 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 void setUp() throws Exception @After public void tearDown() throws Exception { - ioPeon.close(); + outputMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -115,29 +117,30 @@ private void checkSerializedSizeAndData(int chunkFactor) throws Exception FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( - ioPeon, "test", chunkFactor, byteOrder, compressionStrategy + outputMedium, "test", chunkFactor, byteOrder, compressionStrategy ); CompressedIntsIndexedSupplier supplierFromList = CompressedIntsIndexedSupplier.fromList( - Ints.asList(vals), chunkFactor, byteOrder, compressionStrategy + IntArrayList.wrap(vals), + chunkFactor, + byteOrder, + compressionStrategy, + outputMedium.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 OutputBytes outputBytes = outputMedium.makeOutputBytes(); + writer.writeTo(outputBytes, 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(outputBytes.asInputStream())), + byteOrder ); IndexedInts indexedInts = supplierFromByteBuffer.get(); assertEquals(vals.length, indexedInts.size()); @@ -187,51 +190,43 @@ private void checkV2SerializedSizeAndData(int chunkFactor) throws Exception )).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 - ) - ); - - 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(); + + CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( + outputMedium, + chunkFactor, + byteOrder, + compressionStrategy, + GenericIndexedWriter.ofCompressedByteBuffers( + outputMedium, + "test", + compressionStrategy, + Longs.BYTES * 10000 + ) + ); + + 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 3583706973d4..f0c970fa2a65 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.output.OffHeapMemoryOutputMedium; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.Assert; import org.junit.Test; @@ -32,7 +32,6 @@ 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 static Iterable compressionStrategies() { 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 static Iterable compressionStrategies() } 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 @@ private static long[] addUniques(long[] val) public CompressedLongsSerdeTest( CompressionFactory.LongEncodingStrategy encodingStrategy, - CompressedObjectStrategy.CompressionStrategy compressionStrategy, + CompressionStrategy compressionStrategy, ByteOrder order ) { @@ -130,8 +129,12 @@ public void testWithValues(long[] values) throws Exception public void testValues(long[] values) throws Exception { - LongSupplierSerializer serializer = CompressionFactory.getLongSerializer(new IOPeonForTesting(), "test", order, - encodingStrategy, compressionStrategy + LongSupplierSerializer serializer = CompressionFactory.getLongSerializer( + new OffHeapMemoryOutputMedium(), + "test", + order, + encodingStrategy, + compressionStrategy ); serializer.open(); @@ -141,16 +144,7 @@ public void testValues(long[] values) throws Exception 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); @@ -203,7 +197,7 @@ private void assertIndexMatchesVals(IndexedLongs indexed, long[] vals) 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); 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 dd9554da4857..d2c7a001cebd 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 @@ 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 @@ */ 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 IndexedInts apply(int[] input) 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 void testSanity() throws Exception 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); 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 bc6bed9cbcfe..1a3490c8a26a 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -31,10 +31,12 @@ 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.output.OffHeapMemoryOutputMedium; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; import io.druid.segment.CompressedVSizeIndexedV3Supplier; 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.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 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,52 @@ private void generateVals(final int totalSize, final int maxValue) throws IOExce private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFactor) throws Exception { FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); - final IndexedMultivalue indexedMultivalue; - try (IOPeon ioPeon = new TmpFileIOPeon()) { + try (OutputMedium outputMedium = new OffHeapMemoryOutputMedium()) { int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( - ioPeon, "offset", offsetChunkFactor, byteOrder, compressionStrategy + outputMedium, "offset", offsetChunkFactor, byteOrder, compressionStrategy ); CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter( - ioPeon, "value", maxValue, valueChunkFactor, byteOrder, compressionStrategy + outputMedium, "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, + outputMedium.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 OutputBytes outputBytes = outputMedium.makeOutputBytes(); + writer.writeTo(outputBytes, smoosher); smoosher.close(); assertEquals(writtenLength, supplierFromIterable.getSerializedSize()); // read from ByteBuffer and check values CompressedVSizeIndexedV3Supplier supplierFromByteBuffer = CompressedVSizeIndexedV3Supplier.fromByteBuffer( - ByteBuffer.wrap(IOUtils.toByteArray(ioPeon.makeInputStream("output"))), + ByteBuffer.wrap(IOUtils.toByteArray(outputBytes.asInputStream())), byteOrder, null ); - 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 +187,6 @@ public void setUp() throws Exception vals = null; } - @After - public void tearDown() throws Exception - { - ioPeon.close(); - } - @Test public void testSmallData() throws Exception { @@ -244,35 +230,28 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF FileSmoosher smoosher = new FileSmoosher(tmpDirectory); int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; - try (IOPeon ioPeon = new TmpFileIOPeon()) { + try (OutputMedium outputMedium = new OffHeapMemoryOutputMedium()) { CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( + outputMedium, offsetChunkFactor, + byteOrder, compressionStrategy, - new GenericIndexedWriter<>( - ioPeon, "offset", - CompressedIntBufferObjectStrategy.getBufferForOrder( - byteOrder, - compressionStrategy, - offsetChunkFactor - ), + GenericIndexedWriter.ofCompressedByteBuffers( + outputMedium, + "offset", + compressionStrategy, Longs.BYTES * 250000 ) ); - GenericIndexedWriter genericIndexed = new GenericIndexedWriter<>( - ioPeon, + GenericIndexedWriter genericIndexed = GenericIndexedWriter.ofCompressedByteBuffers( + outputMedium, "value", - CompressedByteBufferObjectStrategy.getBufferForOrder( - byteOrder, - compressionStrategy, - valueChunkFactor * VSizeIndexedInts.getNumBytesForMax(maxValue) - + CompressedVSizeIntsIndexedSupplier.bufferPadding(VSizeIndexedInts.getNumBytesForMax(maxValue)) - ), + compressionStrategy, Longs.BYTES * 250000 ); CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter( - ioPeon, - "value", + outputMedium, maxValue, valueChunkFactor, byteOrder, @@ -284,13 +263,9 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF 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); 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 d33d7e48383b..c22a58f9e0ff 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.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 Object[] apply(List input) 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 CompressedVSizeIntsIndexedSupplierTest(CompressedObjectStrategy.Compressi @Before public void setUp() throws Exception { + closer = Closer.create(); CloseQuietly.close(indexed); indexed = null; supplier = null; @@ -110,6 +114,7 @@ public void setUp() throws Exception public void tearDown() throws Exception { CloseQuietly.close(indexed); + closer.close(); } private void setupSimple(final int chunkSize) @@ -119,11 +124,12 @@ private void setupSimple(final int chunkSize) 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 @@ private void makeWithSerde(final int chunkSize) throws IOException 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 14afd1b1648f..bb7782ef950f 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.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.output.OffHeapMemoryOutputMedium; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; +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.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 @@ 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 OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); + 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 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 void setUp() throws Exception @After public void tearDown() throws Exception { - ioPeon.close(); + outputMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -112,29 +114,36 @@ private void checkSerializedSizeAndData(int chunkSize) throws Exception FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter( - ioPeon, "test", vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy + outputMedium, + "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, + outputMedium.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 OutputBytes outputBytes = outputMedium.makeOutputBytes(); + writer.writeTo(outputBytes, 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(outputBytes.asInputStream())), + byteOrder ); IndexedInts indexedInts = supplierFromByteBuffer.get(); for (int i = 0; i < vals.length; ++i) { @@ -177,20 +186,18 @@ private void checkV2SerializedSizeAndData(int chunkSize) throws Exception 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( + outputMedium, "test", - CompressedByteBufferObjectStrategy.getBufferForOrder( - byteOrder, - compressionStrategy, - chunkSize * VSizeIndexedInts.getNumBytesForMax(maxValue) - + CompressedVSizeIntsIndexedSupplier.bufferPadding(VSizeIndexedInts.getNumBytesForMax(maxValue)) - ), + compressionStrategy, Longs.BYTES * 10000 ); CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter( - ioPeon, "test", vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, compressionStrategy, + outputMedium, + vals.length > 0 ? Ints.max(vals) : 0, + chunkSize, + byteOrder, + compressionStrategy, genericIndexed ); writer.open(); @@ -198,12 +205,11 @@ private void checkV2SerializedSizeAndData(int chunkSize) throws Exception 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 @@ private void checkV2SerializedSizeAndData(int chunkSize) throws Exception 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 4867cd27689b..3edaaf1e1a45 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java @@ -25,12 +25,16 @@ 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; @@ -52,11 +56,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 +68,9 @@ public Object[] apply(CompressedObjectStrategy.CompressionStrategy compressionSt ); } - protected final CompressedObjectStrategy.CompressionStrategy compressionStrategy; + protected final CompressionStrategy compressionStrategy; - public CompressionStrategyTest(CompressedObjectStrategy.CompressionStrategy compressionStrategy) + public CompressionStrategyTest(CompressionStrategy compressionStrategy) { this.compressionStrategy = compressionStrategy; } @@ -83,12 +87,27 @@ public static void setupClass() 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); + 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); @@ -98,10 +117,10 @@ public void testBasicOperations() @Test public void testOutputSizeKnownOperations() { - 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, originalData.length); + compressionStrategy.getDecompressor().decompress(compressed, compressed.remaining(), output, originalData.length); byte[] checkArray = new byte[DATA_SIZER]; output.get(checkArray); Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); @@ -110,9 +129,10 @@ public void testOutputSizeKnownOperations() @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 +159,10 @@ public void testConcurrency() throws Exception @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); @@ -173,11 +194,12 @@ public void testKnownSizeConcurrency() throws Exception @Override public void run() { - 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); // TODO: Lambdas would be nice here whenever we use Java 8 compressionStrategy.getDecompressor() - .decompress(compressed, compressed.array().length, output, originalData.length); + .decompress(compressed, compressed.remaining(), output, originalData.length); byte[] checkArray = new byte[DATA_SIZER]; output.get(checkArray); Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); 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 f4bfd70bfffb..2671b5479770 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 @@ private GenericIndexed serializeAndDeserialize(GenericIndexed in { 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 e4043e776aa6..000000000000 --- 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 c8ebcf1290b5..9bd87cdba2e4 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 void testSerialization() throws Exception 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 void testSerialization() throws Exception 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 4a0a4a2926ad..d110225570ba 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.output.OffHeapMemoryOutputMedium; +import io.druid.output.OutputBytes; +import io.druid.output.OutputMedium; +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 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 OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); private final Random rand = new Random(0); private int[] vals; @@ -50,7 +52,7 @@ public void setUp() throws Exception @After public void tearDown() throws Exception { - ioPeon.close(); + outputMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -64,28 +66,24 @@ private void generateVals(final int totalSize, final int maxValue) throws IOExce 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(outputMedium, 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(); + OutputBytes outputBytes = outputMedium.makeOutputBytes(); + writer.writeTo(outputBytes, 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(outputBytes.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 ee46c55bde6e..66d0ecbee843 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 VSizeIndexedInts apply(int[] input) 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 58bc2bcec7cd..8e309d5b3237 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.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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; 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.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; @@ -183,8 +184,9 @@ public static Collection makeConstructors() "roaring", new RoaringBitmapSerdeFactory(true) ); - final Map indexMergers = ImmutableMap.of( - "IndexMergerV9", TestHelper.getTestIndexMergerV9() + final Map outputMediumFactories = ImmutableMap.of( + "tmpFile output medium", TmpFileOutputMediumFactory.instance(), + "off-heap memory output medium", OffHeapMemoryOutputMediumFactory.instance() ); final Map>> finishers = ImmutableMap.of( @@ -248,14 +250,15 @@ public void close() throws IOException ); for (Map.Entry bitmapSerdeFactoryEntry : bitmapSerdeFactories.entrySet()) { - for (Map.Entry indexMergerEntry : indexMergers.entrySet()) { - for (Map.Entry>> finisherEntry : finishers.entrySet()) { + for (Map.Entry outputMediumFactoryEntry : outputMediumFactories.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(), + outputMediumFactoryEntry.getKey(), finisherEntry.getKey(), optimize ); @@ -266,7 +269,7 @@ public void close() throws IOException null, null )) - .indexMerger(indexMergerEntry.getValue()); + .outputMediumFactory(outputMediumFactoryEntry.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 f2def054670f..be0303242ec7 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.Intervals; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.granularity.Granularities; +import io.druid.output.OutputMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -65,6 +66,7 @@ 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 SpatialFilterBonusTest(Segment segment) @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 (OutputMediumFactory outputMediumFactory : OutputMediumFactory.builtInFactories()) { + IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + IndexIO indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + 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 @@ private static IncrementalIndex makeIncrementalIndex() throws IOException 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 @@ private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOE 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 @@ private static QueryableIndex makeMergedQueryableIndex(final IndexSpec indexSpec 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 7e8b1871815c..00545a377042 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.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -73,8 +74,8 @@ @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(OffHeapMemoryOutputMediumFactory.instance()); + private static IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.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 @@ private static QueryableIndex makeQueryableIndex(IndexSpec indexSpec) throws IOE 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 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) 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 @@ private static QueryableIndex makeMergedQueryableIndex(IndexSpec indexSpec) 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 2d06530a0a36..7a523233cfa6 100644 --- a/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/incremental/IncrementalIndexAdapterTest.java @@ -22,8 +22,8 @@ import io.druid.segment.IndexSpec; import io.druid.segment.IndexableAdapter; import io.druid.segment.Rowboat; -import io.druid.segment.data.CompressedObjectStrategy; import io.druid.segment.data.CompressionFactory; +import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.ConciseBitmapSerdeFactory; import io.druid.segment.data.IncrementalIndexTest; import io.druid.segment.data.IndexedInts; @@ -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 a05e4e983898..51901f3641dc 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.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.SegmentizerModule; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.segment.IndexIO; import io.druid.segment.column.ColumnConfig; import org.junit.Assert; @@ -42,14 +43,18 @@ public void testFactory() throws IOException 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, + OffHeapMemoryOutputMediumFactory.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 3f2df176c16b..edf1f402907e 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.metamx.common.StringUtils; import io.druid.data.input.InputRow; import io.druid.hll.HyperLogLogCollector; +import io.druid.output.OutputMedium; 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 int compare(HyperLogLogCollector o1, HyperLogLogCollector o2) } @Override - public GenericColumnSerializer getSerializer(IOPeon peon, String metric) + public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String metric) { return LargeColumnSupportedComplexColumnSerializer.createWithColumnSize( - peon, + outputMedium, 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 a7e7b98cf42d..d406968e1a67 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.Smoosh; import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.java.util.common.io.smoosh.SmooshedWriter; +import io.druid.output.OffHeapMemoryOutputMedium; +import io.druid.output.OutputMedium; 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 void testSanity() throws IOException for (int aCase : cases) { File tmpFile = FileUtils.getTempDirectory(); HyperLogLogCollector baseCollector = HyperLogLogCollector.makeLatestCollector(); - try (IOPeon peon = new TmpFileIOPeon(); + try (OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); FileSmoosher v9Smoosher = new FileSmoosher(tmpFile)) { LargeColumnSupportedComplexColumnSerializer serializer = LargeColumnSupportedComplexColumnSerializer - .createWithColumnSize(peon, "test", serde.getObjectStrategy(), columnSize); + .createWithColumnSize(outputMedium, "test", serde.getObjectStrategy(), columnSize); serializer.open(); for (int i = 0; i < aCase; i++) { @@ -74,13 +74,12 @@ public void testSanity() throws IOException 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/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 53b7e2110af8..45f0cf15efc3 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -60,6 +60,7 @@ 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.output.OutputMediumModule; import io.druid.server.initialization.EmitterModule; import io.druid.server.initialization.jetty.JettyServerModule; import io.druid.server.metrics.MetricsModule; @@ -352,6 +353,7 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new AnnouncerModule(), new AWSModule(), new MetricsModule(), + new OutputMediumModule(), 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 5338055a4d8b..b0cf8d4a9cde 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.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.io.Files; +import io.druid.output.OutputMediumFactory; 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.ShardSpec; import org.joda.time.Period; +import javax.annotation.Nullable; import java.io.File; /** @@ -74,7 +76,8 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final File basePersis 0, defaultReportParseExceptions, defaultHandoffConditionTimeout, - defaultAlertTimeout + defaultAlertTimeout, + null ); } @@ -92,6 +95,8 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final File basePersis private final boolean reportParseExceptions; private final long handoffConditionTimeout; private final long alertTimeout; + @Nullable + private final OutputMediumFactory outputMediumFactory; @JsonCreator public RealtimeTuningConfig( @@ -110,7 +115,8 @@ public RealtimeTuningConfig( @JsonProperty("mergeThreadPriority") int mergeThreadPriority, @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, - @JsonProperty("alertTimeout") Long alertTimeout + @JsonProperty("alertTimeout") Long alertTimeout, + @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; @@ -138,6 +144,7 @@ public RealtimeTuningConfig( this.alertTimeout = alertTimeout == null ? defaultAlertTimeout : alertTimeout; Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); + this.outputMediumFactory = outputMediumFactory; } @Override @@ -240,6 +247,14 @@ public long getAlertTimeout() return alertTimeout; } + @Override + @JsonProperty + @Nullable + public OutputMediumFactory getOutputMediumFactory() + { + return outputMediumFactory; + } + public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) { return new RealtimeTuningConfig( @@ -257,7 +272,8 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) mergeThreadPriority, reportParseExceptions, handoffConditionTimeout, - alertTimeout + alertTimeout, + outputMediumFactory ); } @@ -278,7 +294,8 @@ public RealtimeTuningConfig withBasePersistDirectory(File dir) mergeThreadPriority, reportParseExceptions, handoffConditionTimeout, - alertTimeout + alertTimeout, + outputMediumFactory ); } } 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 4b2c99f644c0..631e27344d99 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.output.OutputMediumFactory; 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 + OutputMediumFactory getOutputMediumFactory(); } 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 fe3c4886f5fa..88f56d9e7bb8 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 @@ -590,7 +590,8 @@ private DataSegment mergeAndPush(final SegmentIdentifier identifier, final Sink schema.getGranularitySpec().isRollup(), schema.getAggregators(), mergedTarget, - tuningConfig.getIndexSpec() + tuningConfig.getIndexSpec(), + tuningConfig.getOutputMediumFactory() ); // Retry pushing segments because uploading to deep storage might fail especially for cloud storage types @@ -1035,7 +1036,8 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdentifier identif indexToPersist.getIndex(), identifier.getInterval(), new File(persistDir, String.valueOf(indexToPersist.getCount())), - indexSpec + indexSpec, + tuningConfig.getOutputMediumFactory() ); 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 3938d017a923..a68e35fa8d1f 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.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 3fe234db39ae..6e77bcf2bf30 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 @@ -421,7 +421,8 @@ public void doRun() schema.getGranularitySpec().isRollup(), schema.getAggregators(), mergedTarget, - config.getIndexSpec() + config.getIndexSpec(), + config.getOutputMediumFactory() ); // emit merge metrics before publishing segment @@ -931,7 +932,8 @@ protected int persistHydrant( indexToPersist.getIndex(), interval, new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())), - indexSpec + indexSpec, + config.getOutputMediumFactory() ); indexToPersist.swapSegment( 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 ed10455f3370..8f6e35742ad9 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.metamx.emitter.EmittingLogger; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; +import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; import io.druid.segment.TestHelper; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; @@ -37,21 +40,36 @@ 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[] {TmpFileOutputMediumFactory.instance()}, + new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + ); + } + @Rule public final TemporaryFolder tmpFolder = new TemporaryFolder(); private final ObjectMapper jsonMapper; + private final OutputMediumFactory outputMediumFactory; private File localSegmentCacheFolder; private SegmentLoaderLocalCacheManager manager; - public SegmentLoaderLocalCacheManagerTest() + public SegmentLoaderLocalCacheManagerTest(OutputMediumFactory outputMediumFactory) { jsonMapper = new DefaultObjectMapper(); jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); @@ -61,6 +79,7 @@ public SegmentLoaderLocalCacheManagerTest() new LocalDataSegmentPuller() ) ); + this.outputMediumFactory = outputMediumFactory; } @Before @@ -76,7 +95,7 @@ public void setUp() throws Exception locations.add(locationConfig); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(outputMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -150,7 +169,7 @@ public void testRetrySuccessAtFirstLocation() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(outputMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -203,7 +222,7 @@ public void testRetrySuccessAtSecondLocation() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(outputMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -258,7 +277,7 @@ public void testRetryAllFail() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(outputMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -312,7 +331,7 @@ public void testEmptyToFullOrder() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexIO(outputMediumFactory), 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 91eb6fd58988..91e0fab5d66b 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.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.TestHelper; @@ -109,8 +110,8 @@ public void testSerde() throws Exception null, null, null, - TestHelper.getTestIndexMergerV9(), - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexMergerV9(OffHeapMemoryOutputMediumFactory.instance()), + TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.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 90b1e66f4b24..8796b3ca288b 100644 --- a/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java +++ b/server/src/test/java/io/druid/segment/realtime/RealtimeManagerTest.java @@ -211,6 +211,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException 0, null, null, + null, null ); plumber = new TestPlumber(new Sink( @@ -269,6 +270,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException 0, null, null, + null, null ); @@ -287,6 +289,7 @@ public FirehoseV2 connect(InputRowParser parser, Object arg1) throws IOException 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 3a6dc6dcfb0a..d7b0aa4a6477 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 @@ EasyMock. anyObject(), 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 a18b97851ccd..96840f15c07b 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 @@ -36,6 +36,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; @@ -148,6 +149,7 @@ public AppenderatorTester( 0, null, null, + null, null ); @@ -156,6 +158,7 @@ public AppenderatorTester( indexIO = new IndexIO( objectMapper, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -165,7 +168,7 @@ public int columnCacheSizeBytes() } } ); - indexMerger = new IndexMergerV9(objectMapper, indexIO); + indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemoryOutputMediumFactory.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 f2ebc839d64f..02b43a7310cd 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 @@ -145,6 +145,7 @@ public int columnCacheSizeBytes() 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 0d983788f39f..5403cbfee3e1 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.hll.HyperLogLogCollector; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; +import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; @@ -51,12 +54,17 @@ 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( @@ -87,11 +95,26 @@ public class IngestSegmentFirehoseTest new HyperUniquesAggregatorFactory("unique_hosts", "unique_hosts") ); + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return ImmutableList.of( + new Object[] {TmpFileOutputMediumFactory.instance()}, + new Object[] {OffHeapMemoryOutputMediumFactory.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(OutputMediumFactory outputMediumFactory) + { + indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + } @Test public void testReadFromIndexAndWriteAnotherIndex() throws Exception @@ -204,7 +227,7 @@ private void createTestIndex(File segmentDir) throws Exception 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 414dd8977389..8bb74fa8e7e7 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.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; +import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.output.OutputMediumFactory; +import io.druid.output.TmpFileOutputMediumFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Query; import io.druid.query.QueryRunnerFactory; @@ -83,7 +86,24 @@ @RunWith(Parameterized.class) public class RealtimePlumberSchoolTest { + @Parameterized.Parameters(name = "rejectionPolicy = {0}, outputMediumFactory = {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, OffHeapMemoryOutputMediumFactory.instance()}); + constructors.add(new Object[]{rejectionPolicy, TmpFileOutputMediumFactory.instance()}); + } + return constructors; + } + private final RejectionPolicyFactory rejectionPolicy; + private final OutputMediumFactory outputMediumFactory; private RealtimePlumber plumber; private RealtimePlumberSchool realtimePlumberSchool; private DataSegmentAnnouncer announcer; @@ -98,24 +118,10 @@ public class RealtimePlumberSchoolTest private FireDepartmentMetrics metrics; private File tmpDir; - public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy) + public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, OutputMediumFactory outputMediumFactory) { 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.outputMediumFactory = outputMediumFactory; } @Before @@ -201,6 +207,7 @@ public void setUp() throws Exception 0, false, null, + null, null ); @@ -212,8 +219,8 @@ public void setUp() throws Exception segmentPublisher, handoffNotifierFactory, MoreExecutors.sameThreadExecutor(), - TestHelper.getTestIndexMergerV9(), - TestHelper.getTestIndexIO(), + TestHelper.getTestIndexMergerV9(outputMediumFactory), + TestHelper.getTestIndexIO(outputMediumFactory), 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 b84d154b2f07..626718e72dfb 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 @@ -73,6 +73,7 @@ public void testSwap() throws Exception 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 b83a93bdbe9d..4ea9acde55f2 100644 --- a/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java +++ b/services/src/test/java/io/druid/cli/validate/DruidJsonValidatorTest.java @@ -181,6 +181,7 @@ public Plumber findPlumber( 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 119342966bdf..057b3146cef6 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.ImmutableSet; import io.druid.data.input.InputRow; import io.druid.java.util.common.Intervals; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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.sql.calcite.planner.Calcites; import io.druid.sql.calcite.planner.PlannerConfig; @@ -87,7 +87,7 @@ public void setUp() throws Exception final File tmpDir = temporaryFolder.newFolder(); final QueryableIndex index1 = IndexBuilder.create() .tmpDir(new File(tmpDir, "1")) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) .schema( new IncrementalIndexSchema.Builder() .withMetrics( @@ -103,7 +103,7 @@ public void setUp() throws Exception final QueryableIndex index2 = IndexBuilder.create() .tmpDir(new File(tmpDir, "2")) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .outputMediumFactory(OffHeapMemoryOutputMediumFactory.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 27908c311e0b..dc9ea1f94d82 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 @@ -42,6 +42,7 @@ import io.druid.guice.ExpressionModule; import io.druid.guice.annotations.Json; import io.druid.math.expr.ExprMacroTable; +import io.druid.output.OffHeapMemoryOutputMediumFactory; import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.DruidProcessingConfig; @@ -325,14 +326,14 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmp { final QueryableIndex index1 = IndexBuilder.create() .tmpDir(new File(tmpDir, "1")) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) .schema(INDEX_SCHEMA) .rows(ROWS1) .buildMMappedIndex(); final QueryableIndex index2 = IndexBuilder.create() .tmpDir(new File(tmpDir, "2")) - .indexMerger(TestHelper.getTestIndexMergerV9()) + .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) .schema(INDEX_SCHEMA) .rows(ROWS2) .buildMMappedIndex(); From e4fac0785e8756e3a1c96ceefc8fc3d0a2c305dd Mon Sep 17 00:00:00 2001 From: leventov Date: Mon, 18 Sep 2017 23:29:31 -0500 Subject: [PATCH 02/14] Fix test --- .../GroupByLimitPushDownInsufficientBufferTest.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) 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 61c145e4e387..df142fe974e6 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.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.math.expr.ExprMacroTable; +import io.druid.output.OffHeapMemoryOutputMediumFactory; 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, + OffHeapMemoryOutputMediumFactory.instance(), new ColumnConfig() { @Override @@ -126,7 +128,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); } @@ -210,7 +212,8 @@ public void setup() throws Exception final File fileA = INDEX_MERGER_V9.persist( indexA, new File(tmpDir, "A"), - new IndexSpec() + new IndexSpec(), + OffHeapMemoryOutputMediumFactory.instance() ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -251,7 +254,8 @@ public void setup() throws Exception final File fileB = INDEX_MERGER_V9.persist( indexB, new File(tmpDir, "B"), - new IndexSpec() + new IndexSpec(), + OffHeapMemoryOutputMediumFactory.instance() ); QueryableIndex qindexB = INDEX_IO.loadIndex(fileB); From 51b09a369a192f300e776ec0eea656aec30d265d Mon Sep 17 00:00:00 2001 From: leventov Date: Wed, 20 Sep 2017 18:19:03 -0500 Subject: [PATCH 03/14] Cleanup CompressionStrategy --- .../segment/data/CompressionStrategy.java | 29 +---------- .../segment/data/CompressionStrategyTest.java | 52 ------------------- 2 files changed, 2 insertions(+), 79 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java index 6dbf4046d853..85a48ee1abae 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java @@ -32,7 +32,6 @@ import io.druid.java.util.common.logger.Logger; import io.druid.segment.CompressedPools; import net.jpountz.lz4.LZ4Factory; -import net.jpountz.lz4.LZ4FastDecompressor; import net.jpountz.lz4.LZ4SafeDecompressor; import org.apache.commons.lang.ArrayUtils; @@ -158,18 +157,12 @@ public static CompressionStrategy[] noNoneValues() return (CompressionStrategy[]) ArrayUtils.removeElement(CompressionStrategy.values(), NONE); } - public static interface Decompressor + 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 */ - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out); - - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize); + void decompress(ByteBuffer in, int numBytes, ByteBuffer out); } public static abstract class Compressor @@ -236,11 +229,6 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) 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 @@ -264,11 +252,6 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) } } - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize) - { - decompress(in, numBytes, out); - } } public static class LZFCompressor extends Compressor @@ -303,7 +286,6 @@ public ByteBuffer compress(ByteBuffer in, ByteBuffer out) public static class LZ4Decompressor implements Decompressor { private static final LZ4SafeDecompressor lz4Safe = LZ4Factory.fastestInstance().safeDecompressor(); - private static final LZ4FastDecompressor lz4Fast = LZ4Factory.fastestInstance().fastDecompressor(); private static final LZ4Decompressor defaultDecompressor = new LZ4Decompressor(); @Override @@ -322,13 +304,6 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) out.limit(out.position() + numDecompressedBytes); } - @Override - public void decompress(ByteBuffer in, int numBytes, ByteBuffer out, int decompressedSize) - { - // lz4Fast.decompress does not modify buffer positions - lz4Fast.decompress(in, in.position(), out, out.position(), decompressedSize); - out.limit(out.position() + decompressedSize); - } } public static class LZ4Compressor extends Compressor 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 3edaaf1e1a45..e94d53f2f7ab 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressionStrategyTest.java @@ -21,10 +21,6 @@ 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; @@ -39,12 +35,10 @@ 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; @@ -113,19 +107,6 @@ public void testBasicOperations() Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); } - - @Test - public void testOutputSizeKnownOperations() - { - 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.remaining(), output, originalData.length); - byte[] checkArray = new byte[DATA_SIZER]; - output.get(checkArray); - Assert.assertArrayEquals("Uncompressed data does not match", originalData, checkArray); - } - @Test public void testDirectMemoryOperations() { @@ -177,37 +158,4 @@ public Boolean call() throws Exception 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 compressionOut = compressionStrategy.getCompressor().allocateOutBuffer(originalData.length, closer); - ByteBuffer compressed = compressionStrategy.getCompressor().compress(ByteBuffer.wrap(originalData), compressionOut); - ByteBuffer output = ByteBuffer.allocate(originalData.length); - // TODO: Lambdas would be nice here whenever we use Java 8 - compressionStrategy.getDecompressor() - .decompress(compressed, compressed.remaining(), 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(); - } } From 628169538d3a882b6e443d093c0c11e63c83ccc9 Mon Sep 17 00:00:00 2001 From: leventov Date: Mon, 2 Oct 2017 16:36:22 -0500 Subject: [PATCH 04/14] Javadocs --- .../java/io/druid/output/ByteBufferOutputBytes.java | 3 +++ .../main/java/io/druid/segment/DimensionHandler.java | 1 + .../io/druid/segment/DoubleDimensionMergerV9.java | 2 +- .../io/druid/segment/GenericColumnSerializer.java | 4 ++-- .../main/java/io/druid/segment/serde/Serializer.java | 12 ++++++++++++ 5 files changed, 19 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java b/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java index c45f5787742d..abecaa0ba2f2 100644 --- a/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java +++ b/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java @@ -35,6 +35,9 @@ public abstract class ByteBufferOutputBytes extends OutputBytes { + /** + * 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<>(); diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java index f304b4f8dc26..761fdf890f7a 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -85,6 +85,7 @@ public interface DimensionHandler * See {@link DimensionMergerV9} interface for more information. * * @param indexSpec Specification object for the index merge + * @param outputMedium this OutputMedium object could be used internally in the created merger, if needed * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler * @param progress ProgressIndicator used by the merging process diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java index 26e5cff9de91..b5749c5c5334 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java @@ -60,7 +60,7 @@ public DoubleDimensionMergerV9( } } - protected void setupEncodedValueWriter(OutputMedium outputMedium) throws IOException + private void setupEncodedValueWriter(OutputMedium outputMedium) throws IOException { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); this.serializer = DoubleColumnSerializer.create(outputMedium, dimensionName, metCompression); diff --git a/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java b/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java index 95215a5ec175..a9f35004fab5 100644 --- a/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/GenericColumnSerializer.java @@ -27,7 +27,7 @@ @ExtensionPoint public interface GenericColumnSerializer extends Serializer { - public void open() throws IOException; + void open() throws IOException; - public void serialize(Object obj) throws IOException; + void serialize(Object obj) throws IOException; } diff --git a/processing/src/main/java/io/druid/segment/serde/Serializer.java b/processing/src/main/java/io/druid/segment/serde/Serializer.java index e59da0e971ec..5c20c11b5025 100644 --- a/processing/src/main/java/io/druid/segment/serde/Serializer.java +++ b/processing/src/main/java/io/druid/segment/serde/Serializer.java @@ -24,8 +24,20 @@ 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; } From 43a3b9f60eaf50129471cc659bd79557f93bff54 Mon Sep 17 00:00:00 2001 From: leventov Date: Mon, 2 Oct 2017 17:14:09 -0500 Subject: [PATCH 05/14] Add OutputBytesTest --- .../output/OnHeapMemoryOutputMedium.java | 49 +++++++++++ .../io/druid/output/TmpFileOutputMedium.java | 4 +- .../java/io/druid/output/OutputBytesTest.java | 84 +++++++++++++++++++ 3 files changed, 135 insertions(+), 2 deletions(-) create mode 100644 processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java create mode 100644 processing/src/test/java/io/druid/output/OutputBytesTest.java diff --git a/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java b/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java new file mode 100644 index 000000000000..4410f2c0f611 --- /dev/null +++ b/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.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.output; + +import com.google.common.annotations.VisibleForTesting; +import io.druid.java.util.common.io.Closer; + +import java.io.IOException; + +@VisibleForTesting +final class OnHeapMemoryOutputMedium implements OutputMedium +{ + private final Closer closer = Closer.create(); + + @Override + public OutputBytes makeOutputBytes() throws IOException + { + return new HeapByteBufferOutputBytes(); + } + + @Override + public Closer getCloser() + { + return closer; + } + + @Override + public void close() throws IOException + { + closer.close(); + } +} diff --git a/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java b/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java index 38c3d9338f1c..6cf50a6e2cd7 100644 --- a/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java +++ b/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java @@ -47,9 +47,9 @@ public OutputBytes makeOutputBytes() throws IOException FileChannel ch = FileChannel.open( file.toPath(), StandardOpenOption.READ, - StandardOpenOption.WRITE, - StandardOpenOption.DELETE_ON_CLOSE + StandardOpenOption.WRITE ); + closer.register(file::delete); closer.register(ch); return new FileOutputBytes(file, ch); } diff --git a/processing/src/test/java/io/druid/output/OutputBytesTest.java b/processing/src/test/java/io/druid/output/OutputBytesTest.java new file mode 100644 index 000000000000..4334fc5ea4cc --- /dev/null +++ b/processing/src/test/java/io/druid/output/OutputBytesTest.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.output; + +import com.google.common.io.Files; +import com.google.common.primitives.Ints; +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.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collection; + +@RunWith(Parameterized.class) +public class OutputBytesTest +{ + @Parameterized.Parameters + public static Collection constructorFeeder() throws IOException + { + return Arrays.asList( + new Object[] {new TmpFileOutputMedium(Files.createTempDir())}, + new Object[] {new OffHeapMemoryOutputMedium()}, + new Object[] {new OnHeapMemoryOutputMedium()} + ); + } + + private final OutputMedium outputMedium; + + public OutputBytesTest(OutputMedium outputMedium) + { + this.outputMedium = outputMedium; + } + + @Test + public void testOutputBytes() throws IOException + { + OutputBytes outputBytes = outputMedium.makeOutputBytes(); + + outputBytes.write('1'); + verifyContents(outputBytes, "1"); + + outputBytes.writeInt(Ints.fromBytes((byte) '2', (byte) '3', (byte) '4', (byte) '5')); + verifyContents(outputBytes, "12345"); + + outputBytes.write(new byte[] {'a'}); + verifyContents(outputBytes, "12345a"); + + outputBytes.write(new byte[] {'a', 'b', 'c'}, 1, 1); + verifyContents(outputBytes, "12345ab"); + + ByteBuffer bb = ByteBuffer.wrap(new byte[]{'a', 'b', 'c'}); + bb.position(2); + outputBytes.write(bb); + Assert.assertEquals(3, bb.position()); + verifyContents(outputBytes, "12345abc"); + } + + private void verifyContents(OutputBytes outputBytes, String expected) throws IOException + { + Assert.assertEquals(expected, IOUtils.toString(outputBytes.asInputStream(), StandardCharsets.US_ASCII)); + } +} From 7e10873e523dacfc6f2cbb1b5515b985adaa8db0 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 3 Oct 2017 20:41:08 -0500 Subject: [PATCH 06/14] Address comments --- .../java/io/druid/common/utils/ByteUtils.java | 35 ++++ .../segment/StringDimensionMergerV9.java | 91 +++++----- .../BlockLayoutDoubleSupplierSerializer.java | 35 ++-- .../BlockLayoutFloatSupplierSerializer.java | 39 ++--- .../BlockLayoutLongSupplierSerializer.java | 59 +++---- .../CompressedDoublesIndexedSupplier.java | 8 +- .../data/CompressedFloatsIndexedSupplier.java | 29 ++-- .../data/CompressedIntsIndexedSupplier.java | 26 +-- .../data/CompressedIntsIndexedWriter.java | 50 +++--- .../data/CompressedLongsIndexedSupplier.java | 39 ++--- .../data/CompressedVSizeIndexedV3Writer.java | 3 + .../CompressedVSizeIntsIndexedSupplier.java | 31 ++-- .../CompressedVSizeIntsIndexedWriter.java | 44 +++-- .../segment/data/CompressionFactory.java | 23 +++ .../segment/data/CompressionStrategy.java | 2 +- .../EntireLayoutDoubleSupplierSerializer.java | 29 ++-- .../EntireLayoutFloatSupplierSerializer.java | 26 +-- .../EntireLayoutLongSupplierSerializer.java | 27 +-- .../io/druid/segment/data/GenericIndexed.java | 27 ++- .../segment/data/GenericIndexedWriter.java | 67 ++++---- .../IntermediateLongSupplierSerializer.java | 5 + .../io/druid/segment/data/VSizeIndexed.java | 28 ++-- .../druid/segment/data/VSizeIndexedInts.java | 23 +-- .../segment/data/VSizeIndexedIntsWriter.java | 29 ++-- .../segment/data/VSizeIndexedWriter.java | 35 ++-- .../druid/segment/serde/MetaSerdeHelper.java | 158 ++++++++++++++++++ 26 files changed, 537 insertions(+), 431 deletions(-) create mode 100644 common/src/main/java/io/druid/common/utils/ByteUtils.java create mode 100644 processing/src/main/java/io/druid/segment/serde/MetaSerdeHelper.java 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 000000000000..f377a628e2bf --- /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/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index eb38d6a90e91..43630455cce6 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -65,31 +65,32 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 { 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 List dictionary; - protected String firstDictionaryValue; - protected int dictionarySize; - protected GenericIndexedWriter bitmapWriter; - protected ByteBufferWriter spatialWriter; - protected ArrayList dimConversions; - protected int cardinality = 0; - protected boolean convertMissingValues = false; - protected boolean hasNull = false; - protected MutableBitmap nullRowsBitmap; + private String dimensionName; + private GenericIndexedWriter dictionaryWriter; + /** This field is used only for spatial indexes */ + private List dictionary; + 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 OutputMedium outputMedium; - protected int rowCount = 0; - protected ColumnCapabilities capabilities; - protected List adapters; - protected ProgressIndicator progress; - protected final IndexSpec indexSpec; - protected IndexMerger.DictionaryMergeIterator dictionaryMergeIterator; + 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, @@ -123,14 +124,13 @@ public void writeMergedValueMetadata(List adapters) throws IOE } int numMergeIndex = 0; - Indexed dimValueLookup = null; Indexed[] dimValueLookups = new Indexed[adapters.size() + 1]; for (int i = 0; i < adapters.size(); i++) { Indexed dimValues = (Indexed) adapters.get(i).getDimValueLookup(dimensionName); if (!isNullColumn(dimValues)) { dimHasValues = true; hasNull |= dimValues.indexOf(null) >= 0; - dimValueLookups[i] = dimValueLookup = dimValues; + dimValueLookups[i] = dimValues; numMergeIndex++; } else { dimAbsentFromSomeIndex = true; @@ -148,7 +148,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE */ if (convertMissingValues && !hasNull) { hasNull = true; - dimValueLookups[adapters.size()] = dimValueLookup = EMPTY_STR_DIM_VAL; + dimValueLookups[adapters.size()] = EMPTY_STR_DIM_VAL; numMergeIndex++; } @@ -165,20 +165,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE cardinality = 0; if (numMergeIndex > 1) { dictionaryMergeIterator = new IndexMerger.DictionaryMergeIterator(dimValueLookups, true); - - while (dictionaryMergeIterator.hasNext()) { - String value = dictionaryMergeIterator.next(); - dictionaryWriter.write(value); - value = Strings.emptyToNull(value); - if (dictionarySize == 0) { - firstDictionaryValue = value; - } - if (hasSpatial) { - dictionary.add(value); - } - dictionarySize++; - } - + writeDictionary(() -> dictionaryMergeIterator, hasSpatial); for (int i = 0; i < adapters.size(); i++) { if (dimValueLookups[i] != null && dictionaryMergeIterator.needConversion(i)) { dimConversions.set(i, dictionaryMergeIterator.conversions[i]); @@ -186,17 +173,8 @@ public void writeMergedValueMetadata(List adapters) throws IOE } cardinality = dictionaryMergeIterator.counter; } else if (numMergeIndex == 1) { - for (String value : dimValueLookup) { - dictionaryWriter.write(value); - value = Strings.emptyToNull(value); - if (dictionarySize == 0) { - firstDictionaryValue = value; - } - if (hasSpatial) { - dictionary.add(value); - } - dictionarySize++; - } + Indexed dimValueLookup = dimValueLookups[0]; + writeDictionary(dimValueLookup, hasSpatial); cardinality = dimValueLookup.size(); } @@ -210,6 +188,21 @@ public void writeMergedValueMetadata(List adapters) throws IOE setupEncodedValueWriter(); } + private void writeDictionary(Iterable dictionaryValues, boolean hasSpatial) throws IOException + { + for (String value : dictionaryValues) { + dictionaryWriter.write(value); + value = Strings.emptyToNull(value); + if (dictionarySize == 0) { + firstDictionaryValue = value; + } + if (hasSpatial) { + dictionary.add(value); + } + dictionarySize++; + } + } + protected void setupEncodedValueWriter() throws IOException { final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); 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 c144492f05e5..4d8ee30ed420 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java @@ -19,12 +19,11 @@ package io.druid.segment.data; -import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputMedium; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -34,6 +33,12 @@ public class BlockLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer { + 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; @@ -74,6 +79,9 @@ public int size() @Override public void add(double value) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { endBuffer.rewind(); flattener.write(endBuffer); @@ -88,36 +96,25 @@ public void add(double value) throws IOException public long getSerializedSize() throws IOException { writeEndBuffer(); - return metaSize() + flattener.getSerializedSize(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writeEndBuffer(); - - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(CompressedDoublesIndexedSupplier.version); - meta.putInt(numInserted); - meta.putInt(CompressedPools.BUFFER_SIZE / Double.BYTES); - meta.put(compression.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); flattener.writeTo(channel, smoosher); } private void writeEndBuffer() throws IOException { - if (endBuffer != null && numInserted > 0) { + if (endBuffer != null) { endBuffer.flip(); - flattener.write(endBuffer); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } endBuffer = null; } } - - private int metaSize() - { - return 1 + Ints.BYTES + Ints.BYTES + 1; - } } 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 f452aa759a02..f8a844cfb3e9 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java @@ -19,13 +19,11 @@ package io.druid.segment.data; -import com.google.common.primitives.Floats; -import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputMedium; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -34,6 +32,12 @@ public class BlockLayoutFloatSupplierSerializer implements FloatSupplierSerializer { + 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; @@ -74,9 +78,13 @@ public int size() @Override public void add(float value) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { - endBuffer.clear(); + endBuffer.rewind(); flattener.write(endBuffer); + endBuffer.clear(); } endBuffer.putFloat(value); ++numInserted; @@ -86,36 +94,25 @@ public void add(float value) throws IOException public long getSerializedSize() throws IOException { writeEndBuffer(); - return metaSize() + flattener.getSerializedSize(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writeEndBuffer(); - - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(CompressedFloatsIndexedSupplier.version); - meta.putInt(numInserted); - meta.putInt(CompressedPools.BUFFER_SIZE / Floats.BYTES); - meta.put(compression.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); flattener.writeTo(channel, smoosher); } private void writeEndBuffer() throws IOException { - if (endBuffer != null && numInserted > 0) { + if (endBuffer != null) { endBuffer.flip(); - flattener.write(endBuffer); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } endBuffer = null; } } - - private int metaSize() - { - return 1 + Ints.BYTES + Ints.BYTES + 1; - } } 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 1c1cdd635cf2..e95f1ae77b41 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java @@ -19,11 +19,10 @@ package io.druid.segment.data; -import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputMedium; import io.druid.segment.CompressedPools; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -32,14 +31,18 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer { - private final OutputMedium outputMedium; + 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 int sizePer; private final CompressionFactory.LongEncodingWriter writer; private final GenericIndexedWriter flattener; - private final ByteOrder byteOrder; private final CompressionStrategy compression; private int numInserted = 0; - private int numInsertedForNextFlush = 0; + private int numInsertedForNextFlush; private ByteBuffer endBuffer = null; @@ -51,13 +54,15 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer CompressionStrategy compression ) { - this.outputMedium = outputMedium; this.sizePer = writer.getBlockSize(CompressedPools.BUFFER_SIZE); int bufferSize = writer.getNumBytes(sizePer); this.flattener = GenericIndexedWriter.ofCompressedByteBuffers(outputMedium, filenameBase, compression, bufferSize); - this.byteOrder = byteOrder; this.writer = writer; this.compression = compression; + CompressionStrategy.Compressor compressor = compression.getCompressor(); + endBuffer = compressor.allocateInBuffer(writer.getNumBytes(sizePer), outputMedium.getCloser()).order(byteOrder); + writer.setBuffer(endBuffer); + numInsertedForNextFlush = sizePer; } @Override @@ -75,18 +80,15 @@ public int size() @Override public void add(long value) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (numInserted == numInsertedForNextFlush) { numInsertedForNextFlush += sizePer; - if (endBuffer != null) { - writer.flush(); - endBuffer.flip(); - flattener.write(endBuffer); - endBuffer.clear(); - } else { - CompressionStrategy.Compressor compressor = compression.getCompressor(); - endBuffer = compressor.allocateInBuffer(writer.getNumBytes(sizePer), outputMedium.getCloser()).order(byteOrder); - writer.setBuffer(endBuffer); - } + writer.flush(); + endBuffer.flip(); + flattener.write(endBuffer); + endBuffer.clear(); } writer.write(value); @@ -97,37 +99,26 @@ public void add(long value) throws IOException public long getSerializedSize() throws IOException { writeEndBuffer(); - return metaSize() + flattener.getSerializedSize(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writeEndBuffer(); - - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(CompressedLongsIndexedSupplier.version); - meta.putInt(numInserted); - meta.putInt(sizePer); - writer.putMeta(meta, compression); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); flattener.writeTo(channel, smoosher); } private void writeEndBuffer() throws IOException { - if (endBuffer != null && numInserted > 0) { + if (endBuffer != null) { writer.flush(); endBuffer.flip(); - flattener.write(endBuffer); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } endBuffer = null; } } - - private int metaSize() - { - return 1 + Ints.BYTES + Ints.BYTES + writer.metaSize(); - } } 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 a95665e5ea77..8a6b758c83c0 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java @@ -31,7 +31,7 @@ public class CompressedDoublesIndexedSupplier implements Supplier { public static final byte LZF_VERSION = 0x1; - public static final byte version = 0x2; + public static final byte VERSION = 0x2; private final int totalSize; private final int sizePer; @@ -64,11 +64,11 @@ public static CompressedDoublesIndexedSupplier fromByteBuffer(ByteBuffer buffer, { 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(); CompressionStrategy compression = CompressionStrategy.LZF; - if (versionFromBuffer == version) { + if (versionFromBuffer == VERSION) { byte compressionId = buffer.get(); compression = CompressionStrategy.forId(compressionId); } @@ -102,7 +102,7 @@ public long getSerializedSize() public void writeToChannel(WritableByteChannel channel) throws IOException { - channel.write(ByteBuffer.wrap(new byte[]{version})); + 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()})); 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 4ee324345a3f..53e14d3efc3c 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedFloatsIndexedSupplier.java @@ -23,6 +23,7 @@ import io.druid.io.Channels; import io.druid.java.util.common.IAE; 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; @@ -33,7 +34,13 @@ 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; @@ -70,37 +77,25 @@ public IndexedFloats get() @Override public long getSerializedSize() throws IOException { - return metaSize() + (long) buffer.remaining(); + return metaSerdeHelper.size(this) + (long) buffer.remaining(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(version); - meta.putInt(totalSize); - meta.putInt(sizePer); - meta.put(compression.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); Channels.writeFully(channel, buffer.asReadOnlyBuffer()); } - private int metaSize() - { - return 1 + 4 + 4 + 1; - } - 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(); CompressionStrategy compression = CompressionStrategy.LZF; - if (versionFromBuffer == version) { + if (versionFromBuffer == VERSION) { byte compressionId = buffer.get(); compression = CompressionStrategy.forId(compressionId); } 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 7bab9f22a101..4fa5504d2ee1 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedSupplier.java @@ -24,13 +24,13 @@ import com.google.common.io.Closeables; import com.google.common.primitives.Ints; import io.druid.collections.ResourceHolder; -import io.druid.io.Channels; import io.druid.java.util.common.IAE; import io.druid.java.util.common.guava.CloseQuietly; 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; @@ -45,6 +45,11 @@ public class CompressedIntsIndexedSupplier implements WritableSupplier 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; @@ -100,31 +105,16 @@ public int get(int index) @Override public long getSerializedSize() throws IOException { - return metaSize() + baseIntBuffers.getSerializedSize(); + return metaSerdeHelper.size(this) + baseIntBuffers.getSerializedSize(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION); - meta.putInt(totalSize); - meta.putInt(sizePer); - meta.put(compression.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); baseIntBuffers.writeTo(channel, smoosher); } - private int metaSize() - { - return 1 + // version - 4 + // totalSize - 4 + // sizePer - 1; // compressionId - } - @VisibleForTesting GenericIndexed getBaseIntBuffers() { diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java index 1789fa4e6537..18a27714e6c5 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java @@ -19,12 +19,11 @@ package io.druid.segment.data; -import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputMedium; import io.druid.segment.IndexIO; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -38,6 +37,12 @@ public class CompressedIntsIndexedWriter extends SingleValueIndexedIntsWriter { private static final byte VERSION = CompressedIntsIndexedSupplier.VERSION; + private static final MetaSerdeHelper metaSerdeHelper = MetaSerdeHelper + .firstWriteByte((CompressedIntsIndexedWriter x) -> VERSION) + .writeInt(x -> x.numInserted) + .writeInt(x -> x.chunkFactor) + .writeByte(x -> x.compression.getId()); + public static CompressedIntsIndexedWriter create( final OutputMedium outputMedium, final String filenameBase, @@ -72,7 +77,12 @@ public static CompressedIntsIndexedWriter create( chunkFactor, byteOrder, compression, - GenericIndexedWriter.ofCompressedByteBuffers(outputMedium, filenameBase, compression, chunkFactor * Ints.BYTES) + GenericIndexedWriter.ofCompressedByteBuffers( + outputMedium, + filenameBase, + compression, + chunkFactor * Integer.BYTES + ) ); } @@ -89,7 +99,7 @@ public static CompressedIntsIndexedWriter create( this.flattener = flattener; CompressionStrategy.Compressor compressor = compression.getCompressor(); Closer closer = outputMedium.getCloser(); - this.endBuffer = compressor.allocateInBuffer(chunkFactor * Ints.BYTES, closer).order(byteOrder); + this.endBuffer = compressor.allocateInBuffer(chunkFactor * Integer.BYTES, closer).order(byteOrder); this.numInserted = 0; } @@ -102,9 +112,13 @@ public void open() throws IOException @Override protected void addValue(int val) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { - endBuffer.clear(); + endBuffer.rewind(); flattener.write(endBuffer); + endBuffer.clear(); } endBuffer.putInt(val); numInserted++; @@ -114,39 +128,25 @@ protected void addValue(int val) throws IOException public long getSerializedSize() throws IOException { writeEndBuffer(); - return metaSize() + flattener.getSerializedSize(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writeEndBuffer(); - - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION); - meta.putInt(numInserted); - meta.putInt(chunkFactor); - meta.put(compression.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); flattener.writeTo(channel, smoosher); } private void writeEndBuffer() throws IOException { - if (endBuffer != null && numInserted > 0) { + if (endBuffer != null) { endBuffer.flip(); - flattener.write(endBuffer); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } endBuffer = null; } } - - private int metaSize() - { - return 1 + // version - Ints.BYTES + // numInserted - Ints.BYTES + // chunkFactor - 1; // compression id - } } 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 66fb75bf6b12..d7af33a82c97 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -24,6 +24,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.MetaSerdeHelper; import io.druid.segment.serde.Serializer; import java.io.IOException; @@ -36,8 +37,17 @@ 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 -> x.compression.getId()); private final int totalSize; private final int sizePer; @@ -77,33 +87,16 @@ public IndexedLongs get() @Override public long getSerializedSize() throws IOException { - return metaSize() + (long) buffer.remaining(); + return metaSerdeHelper.size(this) + (long) buffer.remaining(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(version); - meta.putInt(totalSize); - meta.putInt(sizePer); - if (encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT) { - meta.put(compression.getId()); - } else { - meta.put(CompressionFactory.setEncodingFlag(compression.getId())); - meta.put(encoding.getId()); - } - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); Channels.writeFully(channel, buffer.asReadOnlyBuffer()); } - private int metaSize() - { - return 1 + 4 + 4 + 1 + (encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT ? 0 : 1); - } - public static CompressedLongsIndexedSupplier fromByteBuffer( ByteBuffer buffer, ByteOrder order, @@ -112,12 +105,12 @@ public static CompressedLongsIndexedSupplier fromByteBuffer( { 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(); 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()); 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 a97ecc8f817a..0607b777abc5 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java @@ -90,6 +90,9 @@ public void open() throws IOException @Override protected void addValues(IntList vals) throws IOException { + if (lastOffsetWritten) { + throw new IllegalStateException("written out already"); + } if (vals == null) { vals = IntLists.EMPTY_LIST; } 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 02300c92f3fa..8fc8ae155fe6 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedSupplier.java @@ -25,13 +25,14 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Shorts; import io.druid.collections.ResourceHolder; -import io.druid.io.Channels; +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.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; @@ -46,6 +47,13 @@ public class CompressedVSizeIntsIndexedSupplier implements WritableSupplier 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; @@ -126,33 +134,16 @@ public IndexedInts get() @Override public long getSerializedSize() throws IOException { - return metaSize() + baseBuffers.getSerializedSize(); + return metaSerdeHelper.size(this) + baseBuffers.getSerializedSize(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION); - meta.put((byte) numBytes); - meta.putInt(totalSize); - meta.putInt(sizePer); - meta.put(compression.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); baseBuffers.writeTo(channel, smoosher); } - private int metaSize() - { - return 1 + // version - 1 + // numBytes - Ints.BYTES + // totalSize - Ints.BYTES + // sizePer - 1; // compression id - } - @VisibleForTesting GenericIndexed> getBaseBuffers() { diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java index 0645e4285426..3a1ae711e001 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java @@ -20,10 +20,11 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; -import io.druid.io.Channels; +import io.druid.common.utils.ByteUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputMedium; import io.druid.segment.IndexIO; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -37,6 +38,13 @@ public class CompressedVSizeIntsIndexedWriter extends SingleValueIndexedIntsWrit { private static final byte VERSION = CompressedVSizeIntsIndexedSupplier.VERSION; + private static final MetaSerdeHelper 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 OutputMedium outputMedium, final String filenameBase, @@ -124,9 +132,13 @@ public void open() throws IOException @Override protected void addValue(int val) throws IOException { + if (endBuffer == null) { + throw new IllegalStateException("written out already"); + } if (!endBuffer.hasRemaining()) { - endBuffer.clear(); + endBuffer.rewind(); flattener.write(endBuffer); + endBuffer.clear(); } intBuffer.putInt(0, val); if (isBigEndian) { @@ -141,41 +153,25 @@ protected void addValue(int val) throws IOException public long getSerializedSize() throws IOException { writeEndBuffer(); - return metaSize() + flattener.getSerializedSize(); + return metaSerdeHelper.size(this) + flattener.getSerializedSize(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writeEndBuffer(); - - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION); - meta.put((byte) numBytes); - meta.putInt(numInserted); - meta.putInt(chunkFactor); - meta.put(compression.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); flattener.writeTo(channel, smoosher); } private void writeEndBuffer() throws IOException { - if (endBuffer != null && numInserted > 0) { + if (endBuffer != null) { endBuffer.flip(); - flattener.write(endBuffer); + if (endBuffer.remaining() > 0) { + flattener.write(endBuffer); + } endBuffer = null; } } - - private int metaSize() - { - return 1 + // version - 1 + // numBytes - Ints.BYTES + // numInserted - Ints.BYTES + // chunkFactor - 1; // compression id - } } 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 3f092e7a95db..f2544e6ffd7d 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java @@ -27,11 +27,13 @@ import io.druid.java.util.common.StringUtils; import io.druid.output.OutputBytes; import io.druid.output.OutputMedium; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; 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 CompressionStrategy} @@ -254,6 +256,27 @@ public interface LongEncodingWriter 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); diff --git a/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java index 85a48ee1abae..0025128a4bbc 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java @@ -248,7 +248,7 @@ public void decompress(ByteBuffer in, int numBytes, ByteBuffer out) out.flip(); } catch (IOException e) { - log.error(e, "Error decompressing data"); + throw new RuntimeException(e); } } 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 d125e4845deb..c0a428a13ff4 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java @@ -19,12 +19,10 @@ package io.druid.segment.data; -import com.google.common.primitives.Doubles; -import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputBytes; import io.druid.output.OutputMedium; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -34,6 +32,12 @@ public class EntireLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer { + 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 OutputMedium outputMedium; private final ByteBuffer orderBuffer; private OutputBytes valuesOut; @@ -43,7 +47,7 @@ public class EntireLayoutDoubleSupplierSerializer implements DoubleSupplierSeria public EntireLayoutDoubleSupplierSerializer(OutputMedium outputMedium, ByteOrder order) { this.outputMedium = outputMedium; - this.orderBuffer = ByteBuffer.allocate(Doubles.BYTES); + this.orderBuffer = ByteBuffer.allocate(Double.BYTES); orderBuffer.order(order); } @@ -66,31 +70,18 @@ public void add(double value) throws IOException orderBuffer.putDouble(value); valuesOut.write(orderBuffer.array()); ++numInserted; - } @Override public long getSerializedSize() throws IOException { - return metaSize() + valuesOut.size(); + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(CompressedDoublesIndexedSupplier.version); - meta.putInt(numInserted); - meta.putInt(0); - meta.put(CompressionStrategy.NONE.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); valuesOut.writeTo(channel); } - - private int metaSize() - { - return 1 + Ints.BYTES + Ints.BYTES + 1; - } } 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 2b146d4cf5bb..cc39f14ee27b 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java @@ -19,19 +19,23 @@ package io.druid.segment.data; -import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputBytes; import io.druid.output.OutputMedium; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.channels.WritableByteChannel; public class EntireLayoutFloatSupplierSerializer implements FloatSupplierSerializer { + 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 boolean isLittleEndian; private final OutputMedium outputMedium; private OutputBytes valuesOut; @@ -71,25 +75,13 @@ public void add(float value) throws IOException @Override public long getSerializedSize() throws IOException { - return metaSize() + valuesOut.size(); + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(CompressedFloatsIndexedSupplier.version); - meta.putInt(numInserted); - meta.putInt(0); - meta.put(CompressionStrategy.NONE.getId()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); valuesOut.writeTo(channel); } - - private int metaSize() - { - return 1 + Ints.BYTES + Ints.BYTES + 1; - } } 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 2f8d7274c093..4566178c4a8c 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java @@ -19,18 +19,22 @@ package io.druid.segment.data; -import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputBytes; import io.druid.output.OutputMedium; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; -import java.nio.ByteBuffer; 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 CompressionFactory.LongEncodingWriter writer; private final OutputMedium outputMedium; private OutputBytes valuesOut; @@ -67,27 +71,14 @@ public void add(long value) throws IOException public long getSerializedSize() throws IOException { writer.flush(); - return metaSize() + valuesOut.size(); + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writer.flush(); - - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(CompressedLongsIndexedSupplier.version); - meta.putInt(numInserted); - meta.putInt(0); - writer.putMeta(meta, CompressionStrategy.NONE); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); valuesOut.writeTo(channel); } - - private int metaSize() - { - return 1 + Ints.BYTES + Ints.BYTES + writer.metaSize(); - } } 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 5f5efc944864..93bae9f928bb 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -32,6 +32,7 @@ import io.druid.java.util.common.io.smoosh.SmooshedFileMapper; import io.druid.output.HeapByteBufferOutputBytes; 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; @@ -78,6 +79,13 @@ public class GenericIndexed implements Indexed, Serializer 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() @@ -517,7 +525,7 @@ private static GenericIndexed fromIterableVersionOne( private long getSerializedSizeVersionOne() { - return metaSize() + theBuffer.remaining(); + return metaSerdeHelper.size(this) + (long) theBuffer.remaining(); } private T getVersionOne(int index) @@ -538,14 +546,6 @@ private T getVersionOne(int index) return copyBufferAndGet(firstValueBuffer, startOffset, endOffset); } - private int metaSize() - { - return 1 // version byte - + 1 // allowReverseLookup flag - + Ints.BYTES // numBytesUsed - + Ints.BYTES; // numElements - } - private BufferIndexed singleThreadedVersionOne() { final ByteBuffer copyBuffer = firstValueBuffer.asReadOnlyBuffer(); @@ -582,14 +582,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) private void writeToVersionOne(WritableByteChannel channel) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION_ONE); - meta.put(allowReverseLookup ? REVERSE_LOOKUP_ALLOWED : REVERSE_LOOKUP_DISALLOWED); - meta.putInt(theBuffer.remaining() + 4); - meta.putInt(size); - meta.flip(); - - Channels.writeFully(channel, meta); + 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 1d51ada48e0a..4ebe5f888d3b 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; -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.StringUtils; @@ -30,6 +29,7 @@ import io.druid.java.util.common.io.smoosh.SmooshedWriter; import io.druid.output.OutputBytes; import io.druid.output.OutputMedium; +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; @@ -50,6 +50,25 @@ public class GenericIndexedWriter implements Serializer { private static int PAGE_SIZE = 4096; + 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(x -> x.bagSizePower()) + .writeInt(x -> x.numWritten) + .writeInt(x -> x.fileNameByteArray.length) + .writeByteArray(x -> x.fileNameByteArray); + + static GenericIndexedWriter ofCompressedByteBuffers( final OutputMedium outputMedium, final String filenameBase, @@ -187,6 +206,8 @@ public void write(T objectToWrite) throws IOException } ++numWritten; + // 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); @@ -210,9 +231,10 @@ public void write(T objectToWrite) throws IOException public long getSerializedSize() throws IOException { if (requireMultipleFiles) { - return metaSize(); + // for multi-file version (version 2), getSerializedSize() returns number of bytes in meta file. + return multiFileMetaSerdeHelper.size(this); } else { - return metaSize() + headerOut.size() + valuesOut.size(); + return singleFileMetaSerdeHelper.size(this) + headerOut.size() + valuesOut.size(); } } @@ -220,7 +242,7 @@ public long getSerializedSize() throws IOException public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { if (requireMultipleFiles) { - closeMultiFiles(channel, smoosher); + writeToMultiFiles(channel, smoosher); } else { writeToSingleFile(channel); } @@ -242,19 +264,12 @@ private void writeToSingleFile(WritableByteChannel channel) throws IOException numBytesWritten ); - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(GenericIndexed.VERSION_ONE); - meta.put(objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED); - meta.putInt(Ints.checkedCast(numBytesWritten + Integer.BYTES)); - meta.putInt(numWritten); - meta.flip(); - - Channels.writeFully(channel, meta); + singleFileMetaSerdeHelper.writeTo(channel, this); headerOut.writeTo(channel); valuesOut.writeTo(channel); } - private void closeMultiFiles(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + private void writeToMultiFiles(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { Preconditions.checkState( headerOutLong.size() == numWritten, @@ -273,15 +288,7 @@ private void closeMultiFiles(WritableByteChannel channel, FileSmoosher smoosher) } int bagSizePower = bagSizePower(); - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(GenericIndexed.VERSION_TWO); - meta.put(objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED); - meta.putInt(bagSizePower); - meta.putInt(numWritten); - meta.putInt(fileNameByteArray.length); - meta.put(fileNameByteArray); - meta.flip(); - Channels.writeFully(channel, meta); + multiFileMetaSerdeHelper.writeTo(channel, this); long previousValuePosition = 0; int bagSize = 1 << bagSizePower; @@ -374,22 +381,6 @@ private boolean actuallyFits(int powerTwo) throws IOException return true; } - private int metaSize() - { - // for version 2 getSerializedSize() returns number of bytes in meta file. - if (!requireMultipleFiles) { - return 2 + // version and sorted flag - Ints.BYTES + // numBytesWritten - Ints.BYTES; // numWritten - } 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; - } - } - private void writeHeaderLong(FileSmoosher smoosher, int bagSizePower) throws IOException { diff --git a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java index 48ce7b91cdb4..fbf55e033646 100644 --- a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java @@ -80,6 +80,10 @@ public int size() @Override public void add(long value) throws IOException { + //noinspection VariableNotUsedInsideIf + if (delegate != null) { + throw new IllegalStateException("written out already"); + } tempOut.add(value); ++numInserted; if (uniqueValues.size() <= CompressionFactory.MAX_TABLE_SIZE && !uniqueValues.containsKey(value)) { @@ -96,6 +100,7 @@ public void add(long value) throws IOException private void makeDelegate() throws IOException { + //noinspection VariableNotUsedInsideIf if (delegate != null) { return; } 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 f5373933811f..0560fec7b814 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java @@ -20,12 +20,14 @@ 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.ISE; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.HeapByteBufferOutputBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -36,7 +38,13 @@ */ 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) { @@ -150,28 +158,16 @@ public int indexOf(IndexedInts value) @Override public long getSerializedSize() throws IOException { - return metaSize() + (long) theBuffer.remaining(); + return metaSerdeHelper.size(this) + (long) theBuffer.remaining(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(version); - meta.put((byte) numBytes); - meta.putInt(theBuffer.remaining() + 4); - meta.putInt(size); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); Channels.writeFully(channel, theBuffer.asReadOnlyBuffer()); } - private int metaSize() - { - return 1 + 1 + Ints.BYTES + Ints.BYTES; - } - @Override public IndexedMultivalue get() { @@ -182,7 +178,7 @@ 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(); 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 c4b7bea3b53f..b79149aa816a 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -20,11 +20,13 @@ 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.output.HeapByteBufferOutputBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import io.druid.segment.serde.MetaSerdeHelper; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntList; import it.unimi.dsi.fastutil.ints.IntLists; @@ -39,6 +41,11 @@ public class VSizeIndexedInts implements IndexedInts, Comparable 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)); @@ -166,28 +173,16 @@ public int getNumBytes() @Override public long getSerializedSize() throws IOException { - return metaSize() + buffer.remaining(); + return metaSerdeHelper.size(this) + buffer.remaining(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION); - meta.put((byte) numBytes); - meta.putInt(buffer.remaining()); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); Channels.writeFully(channel, buffer.asReadOnlyBuffer()); } - private int metaSize() - { - // version, numBytes, size - return 1 + 1 + Ints.BYTES; - } - @Override public IndexedInts get() { 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 2b3c5d9cfa9d..33d75b05e327 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java @@ -20,10 +20,11 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; -import io.druid.io.Channels; +import io.druid.common.utils.ByteUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputBytes; import io.druid.output.OutputMedium; +import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; import java.nio.ByteBuffer; @@ -36,6 +37,11 @@ public class VSizeIndexedIntsWriter extends SingleValueIndexedIntsWriter { private static final byte VERSION = VSizeIndexedInts.VERSION; + 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 OutputMedium outputMedium; private final int numBytes; @@ -58,6 +64,9 @@ public void open() throws IOException @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); } @@ -66,21 +75,14 @@ protected void addValue(int val) throws IOException public long getSerializedSize() throws IOException { writeBufPadding(); - return metaSize() + valuesOut.size(); + return metaSerdeHelper.size(this) + valuesOut.size(); } @Override public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException { writeBufPadding(); - - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION); - meta.put((byte) numBytes); - meta.putInt(Ints.checkedCast(valuesOut.size())); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); valuesOut.writeTo(channel); } @@ -92,11 +94,4 @@ private void writeBufPadding() throws IOException bufPaddingWritten = true; } } - - private int metaSize() - { - return 1 + // version - 1 + // numBytes - Ints.BYTES; // dataLen - } } 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 983a174abc64..31ff83fe8e98 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java @@ -21,14 +21,13 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; -import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.output.OutputBytes; import io.druid.output.OutputMedium; +import io.druid.segment.serde.MetaSerdeHelper; import it.unimi.dsi.fastutil.ints.IntList; import java.io.IOException; -import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; /** @@ -38,6 +37,12 @@ public class VSizeIndexedWriter extends MultiValueIndexedIntsWriter { private static final byte VERSION = 0x1; + 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 { @@ -101,6 +106,9 @@ public void open() throws IOException @Override protected void addValues(IntList ints) throws IOException { + if (numBytesForMaxWritten) { + throw new IllegalStateException("written out already"); + } if (ints != null) { for (int i = 0; i < ints.size(); i++) { int value = ints.getInt(i); @@ -116,7 +124,7 @@ protected void addValues(IntList ints) throws IOException public long getSerializedSize() throws IOException { writeNumBytesForMax(); - return metaSize() + headerOut.size() + valuesOut.size(); + return metaSerdeHelper.size(this) + headerOut.size() + valuesOut.size(); } @Override @@ -134,17 +142,12 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I 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 ); - ByteBuffer meta = ByteBuffer.allocate(metaSize()); - meta.put(VERSION); - meta.put(VSizeIndexedInts.getNumBytesForMax(maxId)); - meta.putInt((int) numBytesWritten + 4); - meta.putInt(numWritten); - meta.flip(); - - Channels.writeFully(channel, meta); + metaSerdeHelper.writeTo(channel, this); headerOut.writeTo(channel); valuesOut.writeTo(channel); } @@ -157,12 +160,4 @@ private void writeNumBytesForMax() throws IOException numBytesForMaxWritten = true; } } - - private int metaSize() - { - return 1 + // version - 1 + // numBytes - 4 + // numBytesWritten - 4; // numElements - } } 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 000000000000..0e8c6987d389 --- /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; + } + } +} From 98d602a815bd3a1ba7a7a0aca1247b9667715f26 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 3 Oct 2017 23:25:26 -0500 Subject: [PATCH 07/14] Random access in OutputBytes and GenericIndexedWriter --- .../druid/output/ByteBufferOutputBytes.java | 34 +++++++++++ .../java/io/druid/output/FileOutputBytes.java | 15 +++++ .../output/OnHeapMemoryOutputMedium.java | 2 +- .../java/io/druid/output/OutputBytes.java | 13 +++- .../segment/StringDimensionMergerV9.java | 17 ++---- .../io/druid/segment/data/GenericIndexed.java | 3 +- .../segment/data/GenericIndexedWriter.java | 36 ++++++++++- .../java/io/druid/output/OutputBytesTest.java | 21 +++++++ .../data/GenericIndexedStringWriterTest.java | 59 +++++++++++++++++++ 9 files changed, 183 insertions(+), 17 deletions(-) create mode 100644 processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java diff --git a/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java b/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java index abecaa0ba2f2..985bb40bf581 100644 --- a/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java +++ b/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java @@ -24,9 +24,11 @@ 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; @@ -199,6 +201,38 @@ public void writeTo(ByteBuffer out) } } + @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 { diff --git a/processing/src/main/java/io/druid/output/FileOutputBytes.java b/processing/src/main/java/io/druid/output/FileOutputBytes.java index 77898890707d..b73b5b1913e3 100644 --- a/processing/src/main/java/io/druid/output/FileOutputBytes.java +++ b/processing/src/main/java/io/druid/output/FileOutputBytes.java @@ -21,11 +21,13 @@ 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; @@ -120,6 +122,19 @@ public void writeTo(WritableByteChannel channel) throws IOException } } + @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 { diff --git a/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java b/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java index 4410f2c0f611..ebd36973b491 100644 --- a/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java +++ b/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java @@ -25,7 +25,7 @@ import java.io.IOException; @VisibleForTesting -final class OnHeapMemoryOutputMedium implements OutputMedium +public final class OnHeapMemoryOutputMedium implements OutputMedium { private final Closer closer = Closer.create(); diff --git a/processing/src/main/java/io/druid/output/OutputBytes.java b/processing/src/main/java/io/druid/output/OutputBytes.java index 6d3b354ac7af..89220a4ee318 100644 --- a/processing/src/main/java/io/druid/output/OutputBytes.java +++ b/processing/src/main/java/io/druid/output/OutputBytes.java @@ -22,12 +22,14 @@ 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. + * #writeTo(WritableByteChannel)} and {@link #asInputStream()} allow to write the sequence somewhere else. {@link + * #readFully} allows to access the sequence randomly. * * OutputBytes is a resource that is managed by {@link OutputMedium}, so it's own {@link #close()} method does nothing. * However OutputBytes should appear closed, i. e. {@link #isOpen()} returns false, after the parental OutputMedium is @@ -56,6 +58,15 @@ public abstract class OutputBytes extends OutputStream implements WritableByteCh */ public abstract InputStream asInputStream() throws IOException; + /** + * Reads bytes from the byte sequences, represented by this OutputBytes, at the random position, into the given + * buffer. + * + * @throws RuntimeException if the byte sequences from the given pos ends before all bytes are read + * @throws IllegalArgumentException if the given pos is negative + */ + public abstract void readFully(long pos, ByteBuffer buffer) throws IOException; + /** * @deprecated does nothing. */ diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index 43630455cce6..6fc96cd14406 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -73,8 +73,6 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 private String dimensionName; private GenericIndexedWriter dictionaryWriter; - /** This field is used only for spatial indexes */ - private List dictionary; private String firstDictionaryValue; private int dictionarySize; private GenericIndexedWriter bitmapWriter; @@ -154,10 +152,6 @@ public void writeMergedValueMetadata(List adapters) throws IOE String dictFilename = StringUtils.format("%s.dim_values", dimensionName); dictionaryWriter = new GenericIndexedWriter<>(outputMedium, dictFilename, GenericIndexed.STRING_STRATEGY); - boolean hasSpatial = capabilities.hasSpatialIndexes(); - if (hasSpatial) { - dictionary = new ArrayList<>(); - } firstDictionaryValue = null; dictionarySize = 0; dictionaryWriter.open(); @@ -165,7 +159,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE cardinality = 0; if (numMergeIndex > 1) { dictionaryMergeIterator = new IndexMerger.DictionaryMergeIterator(dimValueLookups, true); - writeDictionary(() -> dictionaryMergeIterator, hasSpatial); + writeDictionary(() -> dictionaryMergeIterator); for (int i = 0; i < adapters.size(); i++) { if (dimValueLookups[i] != null && dictionaryMergeIterator.needConversion(i)) { dimConversions.set(i, dictionaryMergeIterator.conversions[i]); @@ -174,7 +168,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE cardinality = dictionaryMergeIterator.counter; } else if (numMergeIndex == 1) { Indexed dimValueLookup = dimValueLookups[0]; - writeDictionary(dimValueLookup, hasSpatial); + writeDictionary(dimValueLookup); cardinality = dimValueLookup.size(); } @@ -188,7 +182,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE setupEncodedValueWriter(); } - private void writeDictionary(Iterable dictionaryValues, boolean hasSpatial) throws IOException + private void writeDictionary(Iterable dictionaryValues) throws IOException { for (String value : dictionaryValues) { dictionaryWriter.write(value); @@ -196,9 +190,6 @@ private void writeDictionary(Iterable dictionaryValues, boolean hasSpati if (dictionarySize == 0) { firstDictionaryValue = value; } - if (hasSpatial) { - dictionary.add(value); - } dictionarySize++; } } @@ -386,7 +377,7 @@ void mergeBitmaps( bitmapWriter.write(bmpFactory.makeImmutableBitmap(mergedIndexes)); if (hasSpatial) { - String dimVal = dictionary.get(dictId); + String dimVal = dictionaryWriter.get(dictId); if (dimVal != null) { List stringCoords = Lists.newArrayList(SPLITTER.split(dimVal)); float[] coords = new float[stringCoords.size()]; 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 93bae9f928bb..46eaa8369941 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -19,6 +19,7 @@ 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; @@ -105,7 +106,7 @@ public String fromByteBuffer(final ByteBuffer buffer, final int numBytes) @Override public byte[] toBytes(String val) { - if (val == null) { + if (Strings.isNullOrEmpty(val)) { return ByteArrays.EMPTY_ARRAY; } return StringUtils.toUtf8(val); 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 4ebe5f888d3b..8a0482a81611 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -34,6 +34,7 @@ import it.unimi.dsi.fastutil.longs.LongArrayList; import it.unimi.dsi.fastutil.longs.LongList; +import javax.annotation.Nullable; import java.io.DataInput; import java.io.DataInputStream; import java.io.IOException; @@ -63,7 +64,7 @@ public class GenericIndexedWriter implements Serializer .writeByte( x -> x.objectsSorted ? GenericIndexed.REVERSE_LOOKUP_ALLOWED : GenericIndexed.REVERSE_LOOKUP_DISALLOWED ) - .writeInt(x -> x.bagSizePower()) + .writeInt(GenericIndexedWriter::bagSizePower) .writeInt(x -> x.numWritten) .writeInt(x -> x.fileNameByteArray.length) .writeByteArray(x -> x.fileNameByteArray); @@ -144,6 +145,8 @@ public int compare(ByteBuffer o1, ByteBuffer o2) private boolean requireMultipleFiles = false; private LongList headerOutLong; + private final ByteBuffer getOffsetBuffer = ByteBuffer.allocate(Integer.BYTES); + public GenericIndexedWriter(OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy) { this(outputMedium, filenameBase, strategy, Integer.MAX_VALUE & ~PAGE_SIZE); @@ -227,6 +230,37 @@ public void write(T objectToWrite) throws IOException } } + @Nullable + public T get(int index) throws IOException + { + 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 long getSerializedSize() throws IOException { diff --git a/processing/src/test/java/io/druid/output/OutputBytesTest.java b/processing/src/test/java/io/druid/output/OutputBytesTest.java index 4334fc5ea4cc..27b2c0c9fa5b 100644 --- a/processing/src/test/java/io/druid/output/OutputBytesTest.java +++ b/processing/src/test/java/io/druid/output/OutputBytesTest.java @@ -21,6 +21,7 @@ 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; @@ -77,8 +78,28 @@ public void testOutputBytes() throws IOException verifyContents(outputBytes, "12345abc"); } + @Test + public void testCrossBufferRandomAccess() throws IOException + { + OutputBytes outputBytes = outputMedium.makeOutputBytes(); + for (int i = 0; i < ByteBufferOutputBytes.BUFFER_SIZE; i++) { + outputBytes.write('0'); + } + outputBytes.write('1'); + outputBytes.write('2'); + outputBytes.write('3'); + ByteBuffer bb = ByteBuffer.allocate(4); + outputBytes.readFully(ByteBufferOutputBytes.BUFFER_SIZE - 1, bb); + bb.flip(); + Assert.assertEquals("0123", StringUtils.fromUtf8(bb)); + } + private void verifyContents(OutputBytes outputBytes, String expected) throws IOException { Assert.assertEquals(expected, IOUtils.toString(outputBytes.asInputStream(), StandardCharsets.US_ASCII)); + ByteBuffer bb = ByteBuffer.allocate((int) outputBytes.size()); + outputBytes.readFully(0, bb); + bb.flip(); + Assert.assertEquals(expected, StringUtils.fromUtf8(bb)); } } 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 000000000000..6f6888852244 --- /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.output.OnHeapMemoryOutputMedium; +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 + { + OnHeapMemoryOutputMedium outputMedium = new OnHeapMemoryOutputMedium(); + GenericIndexedWriter writer = new GenericIndexedWriter<>( + outputMedium, + "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)); + } + } +} From 3e9b6a24bab5a903b2ac53239f560bd8fd91166a Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 3 Oct 2017 23:52:31 -0500 Subject: [PATCH 08/14] Fix bugs --- .../java/io/druid/segment/StringDimensionMergerV9.java | 6 +++--- .../segment/data/CompressedLongsIndexedSupplier.java | 8 +++++++- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index 6fc96cd14406..3d3ae2119dcb 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -122,13 +122,14 @@ public void writeMergedValueMetadata(List adapters) throws IOE } int numMergeIndex = 0; + Indexed dimValueLookup = null; Indexed[] dimValueLookups = new Indexed[adapters.size() + 1]; for (int i = 0; i < adapters.size(); i++) { Indexed dimValues = (Indexed) adapters.get(i).getDimValueLookup(dimensionName); if (!isNullColumn(dimValues)) { dimHasValues = true; hasNull |= dimValues.indexOf(null) >= 0; - dimValueLookups[i] = dimValues; + dimValueLookups[i] = dimValueLookup = dimValues; numMergeIndex++; } else { dimAbsentFromSomeIndex = true; @@ -146,7 +147,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE */ if (convertMissingValues && !hasNull) { hasNull = true; - dimValueLookups[adapters.size()] = EMPTY_STR_DIM_VAL; + dimValueLookups[adapters.size()] = dimValueLookup = EMPTY_STR_DIM_VAL; numMergeIndex++; } @@ -167,7 +168,6 @@ public void writeMergedValueMetadata(List adapters) throws IOE } cardinality = dictionaryMergeIterator.counter; } else if (numMergeIndex == 1) { - Indexed dimValueLookup = dimValueLookups[0]; writeDictionary(dimValueLookup); cardinality = dimValueLookup.size(); } 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 d7af33a82c97..fb5b29ccaa04 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -47,7 +47,13 @@ public class CompressedLongsIndexedSupplier implements Supplier, S x -> x.encoding != CompressionFactory.LEGACY_LONG_ENCODING_FORMAT, x -> CompressionFactory.setEncodingFlag(x.compression.getId()) ) - .writeByte(x -> x.compression.getId()); + .writeByte(x -> { + if (x.encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT) { + return x.compression.getId(); + } else { + return x.encoding.getId(); + } + }); private final int totalSize; private final int sizePer; From baf90e7049c1f7a1fb911f8f0d43e082b039388b Mon Sep 17 00:00:00 2001 From: leventov Date: Wed, 11 Oct 2017 16:12:18 -0500 Subject: [PATCH 09/14] Fixes --- .../CompressedDoublesIndexedSupplier.java | 36 ++----------------- .../data/CompressedLongsIndexedSupplier.java | 6 ++-- 2 files changed, 5 insertions(+), 37 deletions(-) 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 8a6b758c83c0..a4b639958e66 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedDoublesIndexedSupplier.java @@ -20,13 +20,10 @@ package io.druid.segment.data; import com.google.common.base.Supplier; -import com.google.common.primitives.Ints; import io.druid.java.util.common.IAE; -import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.channels.WritableByteChannel; public class CompressedDoublesIndexedSupplier implements Supplier { @@ -34,24 +31,12 @@ public class CompressedDoublesIndexedSupplier implements Supplier supplier; - private final CompressionStrategy compression; - public CompressedDoublesIndexedSupplier( - int totalSize, - int sizePer, - ByteBuffer buffer, - Supplier supplier, - CompressionStrategy compression - ) + public CompressedDoublesIndexedSupplier(int totalSize, Supplier supplier) { this.totalSize = totalSize; - this.sizePer = sizePer; - this.buffer = buffer; this.supplier = supplier; - this.compression = compression; } @Override @@ -81,10 +66,7 @@ public static CompressedDoublesIndexedSupplier fromByteBuffer(ByteBuffer buffer, ); return new CompressedDoublesIndexedSupplier( totalSize, - sizePer, - buffer, - supplier, - compression + supplier ); } throw new IAE("Unknown version[%s]", versionFromBuffer); @@ -94,18 +76,4 @@ public int size() { return totalSize; } - - public long getSerializedSize() - { - return buffer.remaining() + 1 + 4 + 4 + 1; - } - - public void writeToChannel(WritableByteChannel channel) throws IOException - { - channel.write(ByteBuffer.wrap(new byte[]{VERSION})); - channel.write(ByteBuffer.wrap(Ints.toByteArray(totalSize))); - channel.write(ByteBuffer.wrap(Ints.toByteArray(sizePer))); - channel.write(ByteBuffer.wrap(new byte[]{compression.getId()})); - channel.write(buffer.asReadOnlyBuffer()); - } } 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 fb5b29ccaa04..3897561cd5e0 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -48,10 +48,10 @@ public class CompressedLongsIndexedSupplier implements Supplier, S x -> CompressionFactory.setEncodingFlag(x.compression.getId()) ) .writeByte(x -> { - if (x.encoding == CompressionFactory.LEGACY_LONG_ENCODING_FORMAT) { - return x.compression.getId(); - } else { + if (x.encoding != CompressionFactory.LEGACY_LONG_ENCODING_FORMAT) { return x.encoding.getId(); + } else { + return x.compression.getId(); } }); From 9046e7aeaf591b4dc042b3426e85cc6916119fb3 Mon Sep 17 00:00:00 2001 From: leventov Date: Wed, 11 Oct 2017 16:22:05 -0500 Subject: [PATCH 10/14] Test OutputBytes.readFully() --- .../java/io/druid/output/OutputBytes.java | 3 +- .../java/io/druid/output/OutputBytesTest.java | 29 +++++++++++++++---- 2 files changed, 25 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/io/druid/output/OutputBytes.java b/processing/src/main/java/io/druid/output/OutputBytes.java index 89220a4ee318..04a4a63e8352 100644 --- a/processing/src/main/java/io/druid/output/OutputBytes.java +++ b/processing/src/main/java/io/druid/output/OutputBytes.java @@ -62,7 +62,8 @@ public abstract class OutputBytes extends OutputStream implements WritableByteCh * Reads bytes from the byte sequences, represented by this OutputBytes, at the random position, into the given * buffer. * - * @throws RuntimeException if the byte sequences from the given pos ends before all bytes are read + * @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; diff --git a/processing/src/test/java/io/druid/output/OutputBytesTest.java b/processing/src/test/java/io/druid/output/OutputBytesTest.java index 27b2c0c9fa5b..a2bd49e39ec5 100644 --- a/processing/src/test/java/io/druid/output/OutputBytesTest.java +++ b/processing/src/test/java/io/druid/output/OutputBytesTest.java @@ -29,6 +29,7 @@ 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; @@ -78,6 +79,15 @@ public void testOutputBytes() throws IOException verifyContents(outputBytes, "12345abc"); } + private void verifyContents(OutputBytes outputBytes, String expected) throws IOException + { + Assert.assertEquals(expected, IOUtils.toString(outputBytes.asInputStream(), StandardCharsets.US_ASCII)); + ByteBuffer bb = ByteBuffer.allocate((int) outputBytes.size()); + outputBytes.readFully(0, bb); + bb.flip(); + Assert.assertEquals(expected, StringUtils.fromUtf8(bb)); + } + @Test public void testCrossBufferRandomAccess() throws IOException { @@ -94,12 +104,19 @@ public void testCrossBufferRandomAccess() throws IOException Assert.assertEquals("0123", StringUtils.fromUtf8(bb)); } - private void verifyContents(OutputBytes outputBytes, String expected) throws IOException + @Test(expected = BufferUnderflowException.class) + public void testReadFullyUnderflow() throws IOException { - Assert.assertEquals(expected, IOUtils.toString(outputBytes.asInputStream(), StandardCharsets.US_ASCII)); - ByteBuffer bb = ByteBuffer.allocate((int) outputBytes.size()); - outputBytes.readFully(0, bb); - bb.flip(); - Assert.assertEquals(expected, StringUtils.fromUtf8(bb)); + OutputBytes outputBytes = outputMedium.makeOutputBytes(); + outputBytes.write('1'); + outputBytes.readFully(0, ByteBuffer.allocate(2)); + } + + @Test + public void testReadFullyEmptyAtTheEnd() throws IOException + { + OutputBytes outputBytes = outputMedium.makeOutputBytes(); + outputBytes.write('1'); + outputBytes.readFully(1, ByteBuffer.allocate(0)); } } From 345fc0036c086afdd8a0e5ea22665cd7f2df2808 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 17 Oct 2017 13:56:22 -0500 Subject: [PATCH 11/14] Address comments --- .../content/configuration/indexing-service.md | 6 ++++-- .../extensions-core/kafka-ingestion.md | 2 +- docs/content/ingestion/stream-pull.md | 2 +- docs/content/ingestion/tasks.md | 2 +- .../indexing/kafka/KafkaTuningConfig.java | 20 +++++++++---------- .../io/druid/output/OutputMediumModule.java | 2 +- .../indexing/RealtimeTuningConfig.java | 2 +- 7 files changed, 19 insertions(+), 17 deletions(-) diff --git a/docs/content/configuration/indexing-service.md b/docs/content/configuration/indexing-service.md index d6225d304441..a03611ec2c93 100644 --- a/docs/content/configuration/indexing-service.md +++ b/docs/content/configuration/indexing-service.md @@ -355,7 +355,9 @@ This type of medium may do unnecessary disk I/O and requires some disk space to *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. +`-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 OutputMediumFactory 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 @@ -363,4 +365,4 @@ the value from the configuration below is used: |Property|Description|Default| |--------|-----------|-------| -|`druid.defaultOutputMediumFactory`|`tmpFile` or `offHeapMemory`, see explanation above|`tmpFile`| +|`druid.peon.defaultOutputMediumFactory`|`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 c69fe6d9af84..0db6d24f0431 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -129,7 +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)| -|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.defaultOutputMediumFactory` is used)| +|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultOutputMediumFactory` is used)| #### IndexSpec diff --git a/docs/content/ingestion/stream-pull.md b/docs/content/ingestion/stream-pull.md index b4d2d5cd4c54..832f9d04ae06 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -155,7 +155,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)| -|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.defaultOutputMediumFactory` is used)| +|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultOutputMediumFactory` 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 19fc8c87d51b..f7b9c771bbaa 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -146,7 +146,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |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| -|outputMediumFactory|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|Not specified, the value from `druid.defaultOutputMediumFactory` is used|no| +|outputMediumFactory|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|Not specified, the value from `druid.peon.defaultOutputMediumFactory` is used|no| #### IndexSpec 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 14af3e90996a..dc97b114d6ee 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 @@ -52,17 +52,17 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory ) { diff --git a/processing/src/main/java/io/druid/output/OutputMediumModule.java b/processing/src/main/java/io/druid/output/OutputMediumModule.java index d0f7cc8af98f..68744cde0edd 100644 --- a/processing/src/main/java/io/druid/output/OutputMediumModule.java +++ b/processing/src/main/java/io/druid/output/OutputMediumModule.java @@ -28,6 +28,6 @@ public class OutputMediumModule implements Module @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.defaultOutputMediumFactory", OutputMediumFactory.class); + JsonConfigProvider.bind(binder, "druid.peon.defaultOutputMediumFactory", OutputMediumFactory.class); } } 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 b0cf8d4a9cde..e044cfae16bf 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -59,7 +59,7 @@ private static File createNewBasePersistDirectory() } // 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, From 0300e786422f5950b9a6c70cd97c562de7a6d350 Mon Sep 17 00:00:00 2001 From: leventov Date: Thu, 19 Oct 2017 13:54:58 -0500 Subject: [PATCH 12/14] Rename OutputMedium to SegmentWriteOutMedium and OutputBytes to WriteOutBytes --- .../benchmark/FilterPartitionBenchmark.java | 6 +- .../FilteredAggregatorBenchmark.java | 6 +- ...loatCompressionBenchmarkFileGenerator.java | 4 +- .../benchmark/GenericIndexedBenchmark.java | 4 +- .../GroupByTypeInterfaceBenchmark.java | 6 +- ...LongCompressionBenchmarkFileGenerator.java | 4 +- .../benchmark/TopNTypeInterfaceBenchmark.java | 6 +- .../benchmark/datagen/SegmentGenerator.java | 8 +- .../indexing/IndexMergeBenchmark.java | 6 +- .../indexing/IndexPersistBenchmark.java | 6 +- .../benchmark/query/GroupByBenchmark.java | 6 +- .../benchmark/query/SearchBenchmark.java | 6 +- .../benchmark/query/SelectBenchmark.java | 6 +- .../benchmark/query/TimeseriesBenchmark.java | 6 +- .../druid/benchmark/query/TopNBenchmark.java | 6 +- .../content/configuration/indexing-service.md | 10 +-- .../extensions-core/kafka-ingestion.md | 2 +- docs/content/ingestion/stream-pull.md | 2 +- docs/content/ingestion/tasks.md | 2 +- .../theta/SketchMergeComplexMetricSerde.java | 6 +- .../ApproximateHistogramFoldingSerde.java | 6 +- .../sql/QuantileSqlAggregatorTest.java | 4 +- .../indexing/kafka/KafkaTuningConfig.java | 24 +++--- .../KafkaSupervisorTuningConfig.java | 8 +- .../aggregation/variance/VarianceSerde.java | 6 +- .../common/index/YeOldePlumberSchool.java | 4 +- .../indexing/common/task/AppendTask.java | 8 +- ...ConvertSegmentBackwardsCompatibleTask.java | 10 +-- .../common/task/ConvertSegmentTask.java | 38 ++++----- .../common/task/HadoopConverterTask.java | 6 +- .../druid/indexing/common/task/IndexTask.java | 18 ++--- .../druid/indexing/common/task/MergeTask.java | 8 +- .../indexing/common/task/MergeTaskBase.java | 14 ++-- .../common/task/SameIntervalMergeTask.java | 14 ++-- .../io/druid/indexing/common/TestUtils.java | 6 +- .../indexing/common/task/TaskSerdeTest.java | 12 +-- .../java/io/druid/output/OutputMedium.java | 45 ----------- .../hyperloglog/HyperUniquesSerde.java | 6 +- .../io/druid/segment/DimensionHandler.java | 6 +- .../druid/segment/DoubleColumnSerializer.java | 14 ++-- .../druid/segment/DoubleDimensionHandler.java | 6 +- .../segment/DoubleDimensionMergerV9.java | 10 +-- .../druid/segment/FloatColumnSerializer.java | 14 ++-- .../druid/segment/FloatDimensionHandler.java | 6 +- .../druid/segment/FloatDimensionMergerV9.java | 10 +-- .../main/java/io/druid/segment/IndexIO.java | 16 ++-- .../java/io/druid/segment/IndexMerger.java | 16 ++-- .../java/io/druid/segment/IndexMergerV9.java | 77 ++++++++++--------- .../druid/segment/LongColumnSerializer.java | 14 ++-- .../druid/segment/LongDimensionHandler.java | 6 +- .../druid/segment/LongDimensionMergerV9.java | 10 +-- .../druid/segment/StringDimensionHandler.java | 6 +- .../segment/StringDimensionMergerV9.java | 22 +++--- .../BlockLayoutDoubleSupplierSerializer.java | 8 +- .../BlockLayoutFloatSupplierSerializer.java | 8 +- .../BlockLayoutLongSupplierSerializer.java | 8 +- .../druid/segment/data/ByteBufferWriter.java | 18 ++--- .../data/CompressedIntsIndexedWriter.java | 16 ++-- .../data/CompressedVSizeIndexedV3Writer.java | 8 +- .../CompressedVSizeIntsIndexedWriter.java | 16 ++-- .../segment/data/CompressionFactory.java | 28 +++---- .../segment/data/DeltaLongEncodingWriter.java | 4 +- .../EntireLayoutDoubleSupplierSerializer.java | 14 ++-- .../EntireLayoutFloatSupplierSerializer.java | 16 ++-- .../EntireLayoutLongSupplierSerializer.java | 14 ++-- .../io/druid/segment/data/GenericIndexed.java | 6 +- .../segment/data/GenericIndexedWriter.java | 30 ++++---- .../IntermediateLongSupplierSerializer.java | 12 +-- .../segment/data/LongsLongEncodingWriter.java | 4 +- .../io/druid/segment/data/ObjectStrategy.java | 4 +- .../segment/data/TableLongEncodingWriter.java | 4 +- .../io/druid/segment/data/VSizeIndexed.java | 6 +- .../druid/segment/data/VSizeIndexedInts.java | 4 +- .../segment/data/VSizeIndexedIntsWriter.java | 14 ++-- .../segment/data/VSizeIndexedWriter.java | 28 +++---- .../serde/ComplexColumnSerializer.java | 14 ++-- .../segment/serde/ComplexMetricSerde.java | 6 +- ...olumnSupportedComplexColumnSerializer.java | 22 +++--- .../writeout/ByteBufferWriteOutBytes.java} | 12 +-- .../DirectByteBufferWriteOutBytes.java} | 4 +- .../writeout/FileWriteOutBytes.java} | 6 +- .../HeapByteBufferWriteOutBytes.java} | 4 +- .../OffHeapMemorySegmentWriteOutMedium.java} | 12 +-- ...apMemorySegmentWriteOutMediumFactory.java} | 16 ++-- .../OnHeapMemorySegmentWriteOutMedium.java} | 8 +- .../writeout/SegmentWriteOutMedium.java | 47 +++++++++++ .../SegmentWriteOutMediumFactory.java} | 25 +++--- .../SegmentWriteOutMediumModule.java} | 10 ++- .../TmpFileSegmentWriteOutMedium.java} | 10 +-- .../TmpFileSegmentWriteOutMediumFactory.java} | 14 ++-- .../writeout/WriteOutBytes.java} | 18 ++--- .../io/druid/query/DoubleStorageTest.java | 6 +- .../druid/query/MultiValuedDimensionTest.java | 20 ++--- .../aggregation/AggregationTestHelper.java | 18 ++--- ...ByLimitPushDownInsufficientBufferTest.java | 10 +-- ...roupByLimitPushDownMultiNodeMergeTest.java | 6 +- .../groupby/GroupByMultiSegmentTest.java | 6 +- .../java/io/druid/segment/AppendTest.java | 4 +- .../ConciseBitmapIndexMergerV9Test.java | 8 +- .../java/io/druid/segment/EmptyIndexTest.java | 20 ++--- .../java/io/druid/segment/IndexBuilder.java | 22 +++--- .../java/io/druid/segment/IndexIOTest.java | 4 +- .../io/druid/segment/IndexMergerTestBase.java | 10 +-- .../IndexMergerV9CompatibilityTest.java | 16 ++-- .../IndexMergerV9WithSpatialIndexTest.java | 8 +- .../QueryableIndexIndexableAdapterTest.java | 16 ++-- .../RoaringBitmapIndexMergerV9Test.java | 8 +- .../io/druid/segment/SchemalessIndexTest.java | 8 +- .../druid/segment/SchemalessTestFullTest.java | 14 ++-- .../segment/SchemalessTestSimpleTest.java | 6 +- .../java/io/druid/segment/TestHelper.java | 10 +-- .../test/java/io/druid/segment/TestIndex.java | 6 +- .../data/CompressedFloatsSerdeTest.java | 4 +- .../data/CompressedIntsIndexedWriterTest.java | 24 +++--- .../data/CompressedLongsSerdeTest.java | 4 +- .../CompressedVSizeIndexedV3WriterTest.java | 30 ++++---- .../CompressedVSizeIntsIndexedWriterTest.java | 24 +++--- .../data/GenericIndexedStringWriterTest.java | 6 +- .../data/VSizeIndexedIntsWriterTest.java | 18 ++--- .../druid/segment/filter/BaseFilterTest.java | 29 ++++--- .../filter/SpatialFilterBonusTest.java | 8 +- .../segment/filter/SpatialFilterTest.java | 6 +- .../loading/SegmentizerFactoryTest.java | 4 +- .../serde/HyperUniquesSerdeForTest.java | 6 +- ...nSupportedComplexColumnSerializerTest.java | 8 +- .../writeout/WriteOutBytesTest.java} | 74 +++++++++--------- .../druid/initialization/Initialization.java | 4 +- .../indexing/RealtimeTuningConfig.java | 16 ++-- .../appenderator/AppenderatorConfig.java | 4 +- .../appenderator/AppenderatorImpl.java | 4 +- .../realtime/plumber/RealtimePlumber.java | 4 +- .../SegmentLoaderLocalCacheManagerTest.java | 26 +++---- .../segment/realtime/FireDepartmentTest.java | 6 +- .../appenderator/AppenderatorTester.java | 6 +- .../firehose/IngestSegmentFirehoseTest.java | 16 ++-- .../plumber/RealtimePlumberSchoolTest.java | 22 +++--- .../sql/calcite/schema/DruidSchemaTest.java | 6 +- .../druid/sql/calcite/util/CalciteTests.java | 6 +- 138 files changed, 838 insertions(+), 829 deletions(-) delete mode 100644 processing/src/main/java/io/druid/output/OutputMedium.java rename processing/src/main/java/io/druid/{output/ByteBufferOutputBytes.java => segment/writeout/ByteBufferWriteOutBytes.java} (95%) rename processing/src/main/java/io/druid/{output/DirectByteBufferOutputBytes.java => segment/writeout/DirectByteBufferWriteOutBytes.java} (92%) rename processing/src/main/java/io/druid/{output/FileOutputBytes.java => segment/writeout/FileWriteOutBytes.java} (96%) rename processing/src/main/java/io/druid/{output/HeapByteBufferOutputBytes.java => segment/writeout/HeapByteBufferWriteOutBytes.java} (89%) rename processing/src/main/java/io/druid/{output/OffHeapMemoryOutputMedium.java => segment/writeout/OffHeapMemorySegmentWriteOutMedium.java} (77%) rename processing/src/main/java/io/druid/{output/TmpFileOutputMediumFactory.java => segment/writeout/OffHeapMemorySegmentWriteOutMediumFactory.java} (64%) rename processing/src/main/java/io/druid/{output/OnHeapMemoryOutputMedium.java => segment/writeout/OnHeapMemorySegmentWriteOutMedium.java} (83%) create mode 100644 processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMedium.java rename processing/src/main/java/io/druid/{output/OutputMediumFactory.java => segment/writeout/SegmentWriteOutMediumFactory.java} (56%) rename processing/src/main/java/io/druid/{output/OutputMediumModule.java => segment/writeout/SegmentWriteOutMediumModule.java} (79%) rename processing/src/main/java/io/druid/{output/TmpFileOutputMedium.java => segment/writeout/TmpFileSegmentWriteOutMedium.java} (86%) rename processing/src/main/java/io/druid/{output/OffHeapMemoryOutputMediumFactory.java => segment/writeout/TmpFileSegmentWriteOutMediumFactory.java} (66%) rename processing/src/main/java/io/druid/{output/OutputBytes.java => segment/writeout/WriteOutBytes.java} (75%) rename processing/src/test/java/io/druid/{output/OutputBytesTest.java => segment/writeout/WriteOutBytesTest.java} (50%) diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java index 4e9fbee0700a..b409b9cb6d5b 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilterPartitionBenchmark.java @@ -36,7 +36,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.js.JavaScriptConfig; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +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; @@ -133,7 +133,7 @@ public class FilterPartitionBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -143,7 +143,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup diff --git a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java index 3c421214d620..1703c3a2c22c 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FilteredAggregatorBenchmark.java @@ -35,7 +35,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.js.JavaScriptConfig; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -138,7 +138,7 @@ public class FilteredAggregatorBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -148,7 +148,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup diff --git a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java index 41e6edb97a9c..0f1eaf76d349 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java @@ -23,7 +23,7 @@ import io.druid.benchmark.datagen.BenchmarkColumnSchema; import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator; import io.druid.java.util.common.logger.Logger; -import io.druid.output.OffHeapMemoryOutputMedium; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.CompressionStrategy; @@ -147,7 +147,7 @@ public static void main(String[] args) throws IOException, URISyntaxException File dataFile = new File(dir, entry.getKey()); FloatSupplierSerializer writer = CompressionFactory.getFloatSerializer( - new OffHeapMemoryOutputMedium(), + new OffHeapMemorySegmentWriteOutMedium(), "float", ByteOrder.nativeOrder(), compression diff --git a/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java index cd9f10396cac..2d9a60c559e2 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/GenericIndexedBenchmark.java @@ -23,7 +23,7 @@ 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.output.OffHeapMemoryOutputMedium; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import io.druid.segment.data.GenericIndexed; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.ObjectStrategy; @@ -106,7 +106,7 @@ public int compare(byte[] o1, byte[] o2) public void createGenericIndexed() throws IOException { GenericIndexedWriter genericIndexedWriter = new GenericIndexedWriter<>( - new OffHeapMemoryOutputMedium(), + new OffHeapMemorySegmentWriteOutMedium(), "genericIndexedBenchmark", byteArrayStrategy ); diff --git a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java index a6ac5c178cb3..e8160672d7cd 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -46,7 +46,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -154,7 +154,7 @@ public class GroupByTypeInterfaceBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -164,7 +164,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java index a4a1873f1612..2b654e72599c 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmarkFileGenerator.java @@ -23,7 +23,7 @@ import io.druid.benchmark.datagen.BenchmarkColumnSchema; import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator; import io.druid.java.util.common.logger.Logger; -import io.druid.output.OffHeapMemoryOutputMedium; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import io.druid.segment.column.ValueType; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.CompressionStrategy; @@ -140,7 +140,7 @@ public static void main(String[] args) throws IOException, URISyntaxException File dataFile = new File(dir, entry.getKey()); LongSupplierSerializer writer = CompressionFactory.getLongSerializer( - new OffHeapMemoryOutputMedium(), + new OffHeapMemorySegmentWriteOutMedium(), "long", ByteOrder.nativeOrder(), encoding, diff --git a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java index f35fd4b3b1c6..71645ddcb2eb 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -37,7 +37,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -137,7 +137,7 @@ public class TopNTypeInterfaceBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -147,7 +147,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); 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 67a986301380..dd5ce6629ee9 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java +++ b/benchmarks/src/main/java/io/druid/benchmark/datagen/SegmentGenerator.java @@ -32,7 +32,7 @@ import io.druid.hll.HyperLogLogHash; import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexBuilder; @@ -142,8 +142,8 @@ public QueryableIndex generate( return Iterables.getOnlyElement(indexes); } else { try { - final QueryableIndex merged = TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()).loadIndex( - TestHelper.getTestIndexMergerV9(OffHeapMemoryOutputMediumFactory.instance()).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() @@ -184,7 +184,7 @@ private QueryableIndex makeIndex( .create() .schema(indexSchema) .tmpDir(new File(new File(tempDir, identifier), String.valueOf(indexNumber))) - .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) + .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 abe004bbd554..6f063b8a5360 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -28,7 +28,7 @@ import io.druid.hll.HyperLogLogHash; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.logger.Logger; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; @@ -91,7 +91,7 @@ public class IndexMergeBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -101,7 +101,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup 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 afc1db3d0f3a..1208b7864cc2 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -28,7 +28,7 @@ import io.druid.hll.HyperLogLogHash; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.logger.Logger; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; @@ -89,7 +89,7 @@ public class IndexPersistBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -99,7 +99,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @Setup 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 cd6fe2c6cb08..ad8041d3dfa2 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/GroupByBenchmark.java @@ -47,7 +47,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -159,7 +159,7 @@ public class GroupByBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -169,7 +169,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); 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 50b1d9a7ed9d..8b071f94bd25 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SearchBenchmark.java @@ -38,7 +38,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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.Druids.SearchQueryBuilder; import io.druid.query.FinalizeResultsQueryRunner; @@ -141,7 +141,7 @@ public class SearchBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -151,7 +151,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); 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 98f662ff07d7..f609484c3d30 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/SelectBenchmark.java @@ -37,7 +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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -135,7 +135,7 @@ public class SelectBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -145,7 +145,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); 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 1b722c2fc3cb..a7aa4eb2ea65 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TimeseriesBenchmark.java @@ -35,7 +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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; @@ -133,7 +133,7 @@ public class TimeseriesBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -143,7 +143,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); 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 4ef8886a5538..f213c91808ac 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/query/TopNBenchmark.java @@ -36,7 +36,7 @@ import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; import io.druid.offheap.OffheapBufferGenerator; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -134,7 +134,7 @@ public class TopNBenchmark JSON_MAPPER = new DefaultObjectMapper(); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -144,7 +144,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); diff --git a/docs/content/configuration/indexing-service.md b/docs/content/configuration/indexing-service.md index a03611ec2c93..1ce3152ef582 100644 --- a/docs/content/configuration/indexing-service.md +++ b/docs/content/configuration/indexing-service.md @@ -344,7 +344,7 @@ If the peon is running in remote mode, there must be an overlord up and running. |`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| -##### OutputMediumFactory +##### 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*. @@ -359,10 +359,10 @@ This type of medium is preferred, but it may require to allow the JVM to have mo 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 OutputMediumFactory 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: +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.defaultOutputMediumFactory`|`tmpFile` or `offHeapMemory`, see explanation above|`tmpFile`| +|`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 0db6d24f0431..09e1ecf36a30 100644 --- a/docs/content/development/extensions-core/kafka-ingestion.md +++ b/docs/content/development/extensions-core/kafka-ingestion.md @@ -129,7 +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)| -|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultOutputMediumFactory` is used)| +|`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 832f9d04ae06..71d788c8bbe4 100644 --- a/docs/content/ingestion/stream-pull.md +++ b/docs/content/ingestion/stream-pull.md @@ -155,7 +155,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)| -|outputMediumFactory|String|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultOutputMediumFactory` is used)| +|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 f7b9c771bbaa..a3de44b574b9 100644 --- a/docs/content/ingestion/tasks.md +++ b/docs/content/ingestion/tasks.md @@ -146,7 +146,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |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| -|outputMediumFactory|Output Medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "OutputMediumFactory" for explanation and available options.|Not specified, the value from `druid.peon.defaultOutputMediumFactory` is used|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 8a2cd7c43246..6be52eb405ad 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,7 +21,7 @@ import com.yahoo.sketches.theta.Sketch; import io.druid.data.input.InputRow; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; @@ -80,9 +80,9 @@ public ObjectStrategy getObjectStrategy() } @Override - public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, 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 db739657098e..c5c469e14456 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,7 +21,7 @@ import com.google.common.collect.Ordering; import io.druid.data.input.InputRow; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; @@ -102,9 +102,9 @@ public void deserializeColumn( } @Override - public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, 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 8d7e1b1c1dcf..8d42542d80eb 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,7 +25,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -92,7 +92,7 @@ public void setUp() throws Exception final QueryableIndex index = IndexBuilder.create() .tmpDir(temporaryFolder.newFolder()) - .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) + .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 dc97b114d6ee..429d99ee3737 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 @@ -21,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.RealtimeTuningConfig; import io.druid.segment.indexing.TuningConfig; @@ -48,7 +48,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig private final long handoffConditionTimeout; private final boolean resetOffsetAutomatically; @Nullable - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public KafkaTuningConfig( @@ -63,7 +63,7 @@ public KafkaTuningConfig( @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout, @JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { // Cannot be a static because default basePersistDirectory is unique per-instance @@ -86,7 +86,7 @@ public KafkaTuningConfig( this.resetOffsetAutomatically = resetOffsetAutomatically == null ? DEFAULT_RESET_OFFSET_AUTOMATICALLY : resetOffsetAutomatically; - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } public static KafkaTuningConfig copyOf(KafkaTuningConfig config) @@ -102,7 +102,7 @@ public static KafkaTuningConfig copyOf(KafkaTuningConfig config) config.reportParseExceptions, config.handoffConditionTimeout, config.resetOffsetAutomatically, - config.outputMediumFactory + config.segmentWriteOutMediumFactory ); } @@ -180,9 +180,9 @@ public boolean isResetOffsetAutomatically() @Override @JsonProperty @Nullable - public OutputMediumFactory getOutputMediumFactory() + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() { - return outputMediumFactory; + return segmentWriteOutMediumFactory; } public KafkaTuningConfig withBasePersistDirectory(File dir) @@ -198,7 +198,7 @@ public KafkaTuningConfig withBasePersistDirectory(File dir) reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -215,7 +215,7 @@ public KafkaTuningConfig withMaxRowsInMemory(int rows) reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -238,7 +238,7 @@ public boolean equals(Object o) Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) && Objects.equals(basePersistDirectory, that.basePersistDirectory) && Objects.equals(indexSpec, that.indexSpec) && - Objects.equals(outputMediumFactory, that.outputMediumFactory); + Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory); } @Override @@ -254,7 +254,7 @@ public int hashCode() reportParseExceptions, handoffConditionTimeout, resetOffsetAutomatically, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -271,7 +271,7 @@ public String toString() ", reportParseExceptions=" + reportParseExceptions + ", handoffConditionTimeout=" + handoffConditionTimeout + ", resetOffsetAutomatically=" + resetOffsetAutomatically + - ", outputMediumFactory=" + outputMediumFactory + + ", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory + '}'; } } 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 492f6c8df004..c417df8fbd7c 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,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.indexing.kafka.KafkaTuningConfig; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import org.joda.time.Duration; import org.joda.time.Period; @@ -50,7 +50,7 @@ public KafkaSupervisorTuningConfig( @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, // for backward compatibility @JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("workerThreads") Integer workerThreads, @JsonProperty("chatThreads") Integer chatThreads, @JsonProperty("chatRetries") Long chatRetries, @@ -72,7 +72,7 @@ public KafkaSupervisorTuningConfig( // handoffConditionTimeout handoffConditionTimeout, resetOffsetAutomatically, - outputMediumFactory + segmentWriteOutMediumFactory ); this.workerThreads = workerThreads; @@ -132,7 +132,7 @@ public String toString() ", reportParseExceptions=" + isReportParseExceptions() + ", handoffConditionTimeout=" + getHandoffConditionTimeout() + ", resetOffsetAutomatically=" + isResetOffsetAutomatically() + - ", outputMediumFactory=" + getOutputMediumFactory() + + ", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() + ", workerThreads=" + workerThreads + ", chatThreads=" + chatThreads + ", chatRetries=" + chatRetries + 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 1009a7b20012..cc41bf73bfd8 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,7 +21,7 @@ import com.google.common.collect.Ordering; import io.druid.data.input.InputRow; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; @@ -122,9 +122,9 @@ public int compare(VarianceAggregatorCollector o1, VarianceAggregatorCollector o } @Override - public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } } 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 8c20f1292a7d..938aaef90349 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 @@ -188,7 +188,7 @@ public void finishJob() schema.getAggregators(), fileToUpload, config.getIndexSpec(), - config.getOutputMediumFactory() + config.getSegmentWriteOutMediumFactory() ); } @@ -238,7 +238,7 @@ private void spillIfSwappable() indexToPersist.getIndex(), dirToPersist, config.getIndexSpec(), - config.getOutputMediumFactory() + 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 253eb6a229be..21439b38b827 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,7 +29,7 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.guava.Comparators; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; @@ -64,11 +64,11 @@ public AppendTask( @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { - super(id, dataSource, segments, outputMediumFactory, context); + super(id, dataSource, segments, segmentWriteOutMediumFactory, context); this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.aggregators = aggregators; } @@ -142,7 +142,7 @@ public boolean apply(Rowboat input) aggregators == null ? null : aggregators.toArray(new AggregatorFactory[aggregators.size()]), outDir, indexSpec, - getOutputMediumFactory() + 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 8001eee655ab..ca252c160612 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,7 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -40,7 +40,7 @@ public ConvertSegmentBackwardsCompatibleTask( @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { super( @@ -51,7 +51,7 @@ public ConvertSegmentBackwardsCompatibleTask( indexSpec, force == null ? false : force, validate == null ? false : validate, - outputMediumFactory, + segmentWriteOutMediumFactory, null ); } @@ -66,10 +66,10 @@ public SubTask( @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { - super(groupId, segment, indexSpec, force, validate, outputMediumFactory, 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 3465d37982f5..4605995ee613 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,7 +36,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexIO; import io.druid.segment.IndexSpec; import io.druid.segment.loading.SegmentLoadingException; @@ -80,7 +80,7 @@ public static ConvertSegmentTask create( IndexSpec indexSpec, boolean force, boolean validate, - @Nullable OutputMediumFactory outputMediumFactory, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -93,7 +93,7 @@ public static ConvertSegmentTask create( indexSpec, force, validate, - outputMediumFactory, + segmentWriteOutMediumFactory, context ); } @@ -113,7 +113,7 @@ public static ConvertSegmentTask create( IndexSpec indexSpec, boolean force, boolean validate, - @Nullable OutputMediumFactory outputMediumFactory, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -128,7 +128,7 @@ public static ConvertSegmentTask create( indexSpec, force, validate, - outputMediumFactory, + segmentWriteOutMediumFactory, context ); } @@ -150,16 +150,16 @@ private static ConvertSegmentTask createFromJson( @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, @JsonProperty("context") Map context, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + @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, outputMediumFactory, context); + return create(dataSource, interval, indexSpec, isForce, isValidate, segmentWriteOutMediumFactory, context); } else { - return create(segment, indexSpec, isForce, isValidate, outputMediumFactory, context); + return create(segment, indexSpec, isForce, isValidate, segmentWriteOutMediumFactory, context); } } return new ConvertSegmentTask( @@ -170,7 +170,7 @@ private static ConvertSegmentTask createFromJson( indexSpec, isForce, isValidate, - outputMediumFactory, + segmentWriteOutMediumFactory, context ); } @@ -181,7 +181,7 @@ private static ConvertSegmentTask createFromJson( private final boolean force; private final boolean validate; @Nullable - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; ConvertSegmentTask( String id, @@ -191,7 +191,7 @@ private static ConvertSegmentTask createFromJson( IndexSpec indexSpec, boolean force, boolean validate, - @Nullable OutputMediumFactory outputMediumFactory, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -200,7 +200,7 @@ private static ConvertSegmentTask createFromJson( this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.force = force; this.validate = validate; - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @JsonProperty @@ -235,9 +235,9 @@ public DataSegment getSegment() @JsonProperty @Nullable - public OutputMediumFactory getOutputMediumFactory() + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() { - return outputMediumFactory; + return segmentWriteOutMediumFactory; } @Override @@ -307,7 +307,7 @@ protected Iterable generateSubTasks( @Override public Task apply(DataSegment input) { - return new SubTask(groupId, input, indexSpec, force, validate, outputMediumFactory, context); + return new SubTask(groupId, input, indexSpec, force, validate, segmentWriteOutMediumFactory, context); } } ); @@ -340,7 +340,7 @@ public static class SubTask extends AbstractFixedIntervalTask private final boolean force; private final boolean validate; @Nullable - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public SubTask( @@ -349,7 +349,7 @@ public SubTask( @JsonProperty("indexSpec") IndexSpec indexSpec, @JsonProperty("force") Boolean force, @JsonProperty("validate") Boolean validate, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { @@ -370,7 +370,7 @@ public SubTask( this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; this.force = force == null ? false : force; this.validate = validate == null ? true : validate; - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @JsonProperty @@ -434,7 +434,7 @@ private void convertSegment(TaskToolbox toolbox) throws SegmentLoadingException, 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, outputMediumFactory)) { + 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 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 757f2fdc3115..a8b55715442c 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,7 +34,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import io.druid.timeline.DataSegment; import org.joda.time.Interval; @@ -73,7 +73,7 @@ public HadoopConverterTask( @JsonProperty("jobPriority") String jobPriority, @JsonProperty("segmentOutputPath") String segmentOutputPath, @JsonProperty("classpathPrefix") String classpathPrefix, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { @@ -90,7 +90,7 @@ public HadoopConverterTask( indexSpec, force, validate == null ? true : validate, - outputMediumFactory, + segmentWriteOutMediumFactory, context ); this.hadoopDependencyCoordinates = hadoopDependencyCoordinates; 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 4b8c355de904..840eabb2bf08 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 @@ -58,7 +58,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.DruidMetrics; import io.druid.segment.IndexSpec; import io.druid.segment.indexing.DataSchema; @@ -951,7 +951,7 @@ public static class IndexTuningConfig implements TuningConfig, AppenderatorConfi private final boolean reportParseExceptions; private final long publishTimeout; @Nullable - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public IndexTuningConfig( @@ -968,7 +968,7 @@ public IndexTuningConfig( @JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup, @JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions, @JsonProperty("publishTimeout") @Nullable Long publishTimeout, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { this( @@ -983,7 +983,7 @@ public IndexTuningConfig( reportParseExceptions, publishTimeout, null, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -1004,7 +1004,7 @@ private IndexTuningConfig( @Nullable Boolean reportParseExceptions, @Nullable Long publishTimeout, @Nullable File basePersistDirectory, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { Preconditions.checkArgument( @@ -1039,7 +1039,7 @@ private IndexTuningConfig( "Perfect rollup cannot be guaranteed with extendable shardSpecs" ); - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } public IndexTuningConfig withBasePersistDirectory(File dir) @@ -1056,7 +1056,7 @@ public IndexTuningConfig withBasePersistDirectory(File dir) reportParseExceptions, publishTimeout, dir, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -1149,9 +1149,9 @@ public Period getIntermediatePersistPeriod() @Nullable @Override @JsonProperty - public OutputMediumFactory getOutputMediumFactory() + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() { - return outputMediumFactory; + return 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 b54f4e02a2de..fe9ca37123cc 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,7 +28,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import io.druid.indexing.common.TaskToolbox; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexMerger; import io.druid.segment.IndexSpec; @@ -59,11 +59,11 @@ public MergeTask( @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { - super(id, dataSource, segments, outputMediumFactory, 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; @@ -95,7 +95,7 @@ public QueryableIndex apply(@Nullable File input) aggregators.toArray(new AggregatorFactory[aggregators.size()]), outDir, indexSpec, - getOutputMediumFactory() + 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 3518101d8852..8a6cb91dea52 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,7 +45,7 @@ import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexIO; import io.druid.timeline.DataSegment; import io.druid.timeline.partition.NoneShardSpec; @@ -68,13 +68,13 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask private final List segments; @JsonIgnore @Nullable - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; protected MergeTaskBase( final String id, final String dataSource, final List segments, - final @Nullable OutputMediumFactory outputMediumFactory, + final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -109,7 +109,7 @@ public boolean apply(@Nullable DataSegment segment) verifyInputSegments(segments); this.segments = segments; - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } protected void verifyInputSegments(List segments) @@ -262,9 +262,9 @@ public List getSegments() @JsonProperty @Nullable - public OutputMediumFactory getOutputMediumFactory() + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() { - return outputMediumFactory; + return segmentWriteOutMediumFactory; } @Override @@ -275,7 +275,7 @@ public String toString() .add("dataSource", getDataSource()) .add("interval", getInterval()) .add("segments", segments) - .add("outputMediumFactory", outputMediumFactory) + .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 8a0ac3c4c5cf..b396dbde30d2 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,7 +26,7 @@ import io.druid.indexing.common.TaskToolbox; import io.druid.indexing.common.actions.SegmentListUsedAction; import io.druid.java.util.common.DateTimes; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.IndexSpec; import io.druid.timeline.DataSegment; @@ -46,7 +46,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask private final Boolean rollup; private final IndexSpec indexSpec; @Nullable - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; public SameIntervalMergeTask( @JsonProperty("id") String id, @@ -57,7 +57,7 @@ public SameIntervalMergeTask( @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("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory, + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, @JsonProperty("context") Map context ) { @@ -70,7 +70,7 @@ public SameIntervalMergeTask( this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations"); this.rollup = rollup == null ? Boolean.TRUE : rollup; this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec; - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @JsonProperty("aggregations") @@ -135,7 +135,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception aggregators, rollup, indexSpec, - outputMediumFactory, + segmentWriteOutMediumFactory, getContext() ); final TaskStatus status = mergeTask.run(toolbox); @@ -154,7 +154,7 @@ private SubTask( List aggregators, Boolean rollup, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory, + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, Map context ) { @@ -166,7 +166,7 @@ private SubTask( rollup, indexSpec, true, - outputMediumFactory, + 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 778ece3e17e9..d14425db1e6b 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 @@ -29,7 +29,7 @@ import io.druid.java.util.common.ISE; import io.druid.java.util.common.logger.Logger; import io.druid.math.expr.ExprMacroTable; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.expression.TestExpressionMacroTable; import io.druid.segment.IndexIO; import io.druid.segment.IndexMergerV9; @@ -57,7 +57,7 @@ public TestUtils() jsonMapper = new DefaultObjectMapper(); indexIO = new IndexIO( jsonMapper, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -67,7 +67,7 @@ public int columnCacheSizeBytes() } } ); - indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()); + 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/TaskSerdeTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/TaskSerdeTest.java index 096ccee6cf21..f3f7aae133f4 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,7 +35,7 @@ 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.output.TmpFileOutputMediumFactory; +import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -415,7 +415,7 @@ public void testVersionConverterTaskSerde() throws Exception null, false, true, - TmpFileOutputMediumFactory.instance(), + TmpFileSegmentWriteOutMediumFactory.instance(), null ); @@ -432,7 +432,7 @@ public void testVersionConverterTaskSerde() throws Exception Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getInterval(), task2.getInterval()); Assert.assertEquals(task.getSegment(), task2.getSegment()); - Assert.assertEquals(task.getOutputMediumFactory(), task2.getOutputMediumFactory()); + Assert.assertEquals(task.getSegmentWriteOutMediumFactory(), task2.getSegmentWriteOutMediumFactory()); } @Test @@ -670,7 +670,7 @@ public void testSegmentConvetSerdeReflection() throws IOException indexSpec, false, true, - TmpFileOutputMediumFactory.instance(), + TmpFileSegmentWriteOutMediumFactory.instance(), null ); final String json = jsonMapper.writeValueAsString(task); @@ -702,7 +702,7 @@ public void testSegmentConvertSerde() throws IOException ), false, true, - TmpFileOutputMediumFactory.instance(), + TmpFileSegmentWriteOutMediumFactory.instance(), null ); final String json = jsonMapper.writeValueAsString(originalTask); @@ -728,7 +728,7 @@ public void testSegmentConvertSerde() throws IOException ); Assert.assertEquals(false, convertSegmentTask.isForce()); Assert.assertEquals(segment, convertSegmentTask.getSegment()); - Assert.assertEquals(originalTask.getOutputMediumFactory(), convertSegmentTask.getOutputMediumFactory()); + Assert.assertEquals(originalTask.getSegmentWriteOutMediumFactory(), convertSegmentTask.getSegmentWriteOutMediumFactory()); } @Test diff --git a/processing/src/main/java/io/druid/output/OutputMedium.java b/processing/src/main/java/io/druid/output/OutputMedium.java deleted file mode 100644 index 3b2978b60b5e..000000000000 --- a/processing/src/main/java/io/druid/output/OutputMedium.java +++ /dev/null @@ -1,45 +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.output; - -import io.druid.java.util.common.io.Closer; - -import java.io.Closeable; -import java.io.IOException; - -/** - * OutputMedium manages the resources of a bunch of {@link OutputBytes} objects of the same kind, created by calling - * {@link #makeOutputBytes()} on the OutputMedium object. When OutputMedium is closed, all child OutputBytes couldn't - * be used anymore. - */ -public interface OutputMedium extends Closeable -{ - /** - * Creates a new empty {@link OutputBytes}, attached to this OutputMedium. When this OutputMedium is closed, the - * returned OutputBytes couldn't be used anymore. - */ - OutputBytes makeOutputBytes() throws IOException; - - /** - * Returns a closer of this OutputMedium, which is closed in this OutputMedium's close() method. Could be used to - * "attach" some random resources to this OutputMedium, to be closed at the same time. - */ - Closer getCloser(); -} 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 3a255ff45459..93b5aacea0ab 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,7 +23,7 @@ import io.druid.data.input.InputRow; import io.druid.hll.HyperLogLogCollector; import io.druid.hll.HyperLogLogHash; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; @@ -148,9 +148,9 @@ public int compare(HyperLogLogCollector o1, HyperLogLogCollector o2) } @Override - public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(outputMedium, column, this.getObjectStrategy()); + return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } } diff --git a/processing/src/main/java/io/druid/segment/DimensionHandler.java b/processing/src/main/java/io/druid/segment/DimensionHandler.java index 761fdf890f7a..c73ca7e3953a 100644 --- a/processing/src/main/java/io/druid/segment/DimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DimensionHandler.java @@ -19,7 +19,7 @@ package io.druid.segment; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.data.Indexed; @@ -85,7 +85,7 @@ public interface DimensionHandler * See {@link DimensionMergerV9} interface for more information. * * @param indexSpec Specification object for the index merge - * @param outputMedium this OutputMedium object could be used internally in the created merger, if needed + * @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if needed * @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler * @param progress ProgressIndicator used by the merging process @@ -93,7 +93,7 @@ public interface DimensionHandler */ DimensionMergerV9 makeMerger( IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) throws IOException; diff --git a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java index 7ea1ddd954ec..70979e84a65a 100644 --- a/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/DoubleColumnSerializer.java @@ -21,7 +21,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.DoubleSupplierSerializer; @@ -33,28 +33,28 @@ public class DoubleColumnSerializer implements GenericColumnSerializer { public static DoubleColumnSerializer create( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, CompressionStrategy compression ) { - return new DoubleColumnSerializer(outputMedium, filenameBase, IndexIO.BYTE_ORDER, compression); + return new DoubleColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression); } - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ByteOrder byteOrder; private final CompressionStrategy compression; private DoubleSupplierSerializer writer; private DoubleColumnSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, CompressionStrategy compression ) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -64,7 +64,7 @@ private DoubleColumnSerializer( public void open() throws IOException { writer = CompressionFactory.getDoubleSerializer( - outputMedium, + segmentWriteOutMedium, StringUtils.format("%s.double_column", filenameBase), byteOrder, compression diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java b/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java index b1491d3d2a56..6273586a9c77 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionHandler.java @@ -19,7 +19,7 @@ package io.druid.segment; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.DoubleColumn; @@ -53,7 +53,7 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) throws IOException @@ -61,7 +61,7 @@ public DimensionMergerV9 makeMerger( return new DoubleDimensionMergerV9( dimensionName, indexSpec, - outputMedium, + segmentWriteOutMedium, capabilities, progress ); diff --git a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java index b5749c5c5334..45b6cbf5a4d5 100644 --- a/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/DoubleDimensionMergerV9.java @@ -20,7 +20,7 @@ package io.druid.segment; import io.druid.java.util.common.io.Closer; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; @@ -42,7 +42,7 @@ public class DoubleDimensionMergerV9 implements DimensionMergerV9 public DoubleDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -53,17 +53,17 @@ public DoubleDimensionMergerV9( this.progress = progress; try { - setupEncodedValueWriter(outputMedium); + setupEncodedValueWriter(segmentWriteOutMedium); } catch (IOException ioe) { throw new RuntimeException(ioe); } } - private void setupEncodedValueWriter(OutputMedium outputMedium) throws IOException + private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = DoubleColumnSerializer.create(outputMedium, dimensionName, metCompression); + this.serializer = DoubleColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression); serializer.open(); } diff --git a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java index abc5dffae20b..cf39ddb85cd5 100644 --- a/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/FloatColumnSerializer.java @@ -21,7 +21,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.FloatSupplierSerializer; @@ -33,28 +33,28 @@ public class FloatColumnSerializer implements GenericColumnSerializer { public static FloatColumnSerializer create( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, CompressionStrategy compression ) { - return new FloatColumnSerializer(outputMedium, filenameBase, IndexIO.BYTE_ORDER, compression); + return new FloatColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression); } - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ByteOrder byteOrder; private final CompressionStrategy compression; private FloatSupplierSerializer writer; private FloatColumnSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, CompressionStrategy compression ) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -64,7 +64,7 @@ private FloatColumnSerializer( public void open() throws IOException { writer = CompressionFactory.getFloatSerializer( - outputMedium, + segmentWriteOutMedium, StringUtils.format("%s.float_column", filenameBase), byteOrder, compression diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java index 47385065b5c5..d04d24621e0f 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionHandler.java @@ -19,7 +19,7 @@ package io.druid.segment; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.FloatColumn; @@ -53,7 +53,7 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) throws IOException @@ -61,7 +61,7 @@ public DimensionMergerV9 makeMerger( return new FloatDimensionMergerV9( dimensionName, indexSpec, - outputMedium, + segmentWriteOutMedium, capabilities, progress ); diff --git a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java index fb9bfe6b45b5..344c3781e28d 100644 --- a/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/FloatDimensionMergerV9.java @@ -20,7 +20,7 @@ package io.druid.segment; import io.druid.java.util.common.io.Closer; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; @@ -43,7 +43,7 @@ public class FloatDimensionMergerV9 implements DimensionMergerV9 public FloatDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -54,17 +54,17 @@ public FloatDimensionMergerV9( this.progress = progress; try { - setupEncodedValueWriter(outputMedium); + setupEncodedValueWriter(segmentWriteOutMedium); } catch (IOException ioe) { throw new RuntimeException(ioe); } } - private void setupEncodedValueWriter(OutputMedium outputMedium) throws IOException + private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); - this.serializer = FloatColumnSerializer.create(outputMedium, dimensionName, metCompression); + this.serializer = FloatColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression); serializer.open(); } diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 26c8c6505385..80863b406a9c 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -44,7 +44,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.column.ColumnCapabilities; @@ -97,13 +97,13 @@ public class IndexIO private static final SerializerUtils serializerUtils = new SerializerUtils(); private final ObjectMapper mapper; - private final OutputMediumFactory defaultOutputMediumFactory; + private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory; @Inject - public IndexIO(ObjectMapper mapper, OutputMediumFactory defaultOutputMediumFactory, ColumnConfig columnConfig) + public IndexIO(ObjectMapper mapper, SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory, ColumnConfig columnConfig) { this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); - this.defaultOutputMediumFactory = Preconditions.checkNotNull(defaultOutputMediumFactory, "null OutputMediumFactory"); + this.defaultSegmentWriteOutMediumFactory = Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory"); Preconditions.checkNotNull(columnConfig, "null ColumnConfig"); ImmutableMap.Builder indexLoadersBuilder = ImmutableMap.builder(); LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig); @@ -229,16 +229,16 @@ public boolean convertSegment( IndexSpec indexSpec, boolean forceIfCurrent, boolean validate, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { final int version = SegmentUtils.getVersionFromDir(toConvert); boolean current = version == CURRENT_VERSION_ID; if (!current || forceIfCurrent) { - if (outputMediumFactory == null) { - outputMediumFactory = this.defaultOutputMediumFactory; + if (segmentWriteOutMediumFactory == null) { + segmentWriteOutMediumFactory = this.defaultSegmentWriteOutMediumFactory; } - new IndexMergerV9(mapper, this, outputMediumFactory).convert(toConvert, converted, indexSpec); + new IndexMergerV9(mapper, this, segmentWriteOutMediumFactory).convert(toConvert, converted, indexSpec); if (validate) { validateTwoSegments(toConvert, converted); } diff --git a/processing/src/main/java/io/druid/segment/IndexMerger.java b/processing/src/main/java/io/druid/segment/IndexMerger.java index f5b6c58505b3..8c609e8a27aa 100644 --- a/processing/src/main/java/io/druid/segment/IndexMerger.java +++ b/processing/src/main/java/io/druid/segment/IndexMerger.java @@ -37,7 +37,7 @@ import io.druid.java.util.common.guava.nary.BinaryFn; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.parsers.CloseableIterator; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.ColumnCapabilitiesImpl; import io.druid.segment.data.Indexed; @@ -163,7 +163,7 @@ File persist( IncrementalIndex index, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; /** @@ -183,7 +183,7 @@ File persist( Interval dataInterval, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; File persist( @@ -192,7 +192,7 @@ File persist( File outDir, IndexSpec indexSpec, ProgressIndicator progress, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; File mergeQueryableIndex( @@ -201,7 +201,7 @@ File mergeQueryableIndex( AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; File mergeQueryableIndex( @@ -211,7 +211,7 @@ File mergeQueryableIndex( File outDir, IndexSpec indexSpec, ProgressIndicator progress, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; @VisibleForTesting @@ -230,7 +230,7 @@ File convert( File outDir, IndexSpec indexSpec, ProgressIndicator progress, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; File append( @@ -238,7 +238,7 @@ File append( AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException; interface IndexSeeker diff --git a/processing/src/main/java/io/druid/segment/IndexMergerV9.java b/processing/src/main/java/io/druid/segment/IndexMergerV9.java index b4641e5f90b8..6753eeaa8cd6 100644 --- a/processing/src/main/java/io/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/io/druid/segment/IndexMergerV9.java @@ -45,8 +45,8 @@ import io.druid.java.util.common.io.smoosh.FileSmoosher; import io.druid.java.util.common.io.smoosh.SmooshedWriter; import io.druid.java.util.common.logger.Logger; -import io.druid.output.OutputMedium; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMedium; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; @@ -92,15 +92,15 @@ public class IndexMergerV9 implements IndexMerger private final ObjectMapper mapper; private final IndexIO indexIO; - private final OutputMediumFactory defaultOutputMediumFactory; + private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory; @Inject - public IndexMergerV9(ObjectMapper mapper, IndexIO indexIO, OutputMediumFactory defaultOutputMediumFactory) + public IndexMergerV9(ObjectMapper mapper, IndexIO indexIO, SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory) { this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper"); this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO"); - this.defaultOutputMediumFactory = - Preconditions.checkNotNull(defaultOutputMediumFactory, "null OutputMediumFactory"); + this.defaultSegmentWriteOutMediumFactory = + Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory"); } private File makeIndexFiles( @@ -112,7 +112,7 @@ private File makeIndexFiles( final List mergedMetrics, final Function>, Iterable> rowMergerFn, final IndexSpec indexSpec, - final @Nullable OutputMediumFactory outputMediumFactory + final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { progress.start(); @@ -152,10 +152,11 @@ public Metadata apply(IndexableAdapter input) final FileSmoosher v9Smoosher = new FileSmoosher(outDir); FileUtils.forceMkdir(outDir); - OutputMediumFactory omf = outputMediumFactory != null ? outputMediumFactory : defaultOutputMediumFactory; - log.info("Using OutputMediumFactory[%s]", omf.getClass().getSimpleName()); - OutputMedium outputMedium = omf.makeOutputMedium(outDir); - closer.register(outputMedium); + 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(); Files.asByteSink(new File(outDir, "version.bin")).write(Ints.toByteArray(IndexIO.V9_VERSION)); log.info("Completed version.bin in %,d millis.", System.currentTimeMillis() - startTime); @@ -176,7 +177,7 @@ public Metadata apply(IndexableAdapter input) 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, outputMedium, dimCapabilities.get(i), progress)); + mergers.add(handlers[i].makeMerger(indexSpec, segmentWriteOutMedium, dimCapabilities.get(i), progress)); } /************* Setup Dim Conversions **************/ @@ -196,9 +197,9 @@ public Metadata apply(IndexableAdapter input) handlers, mergers ); - final LongColumnSerializer timeWriter = setupTimeWriter(outputMedium, indexSpec); + final LongColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec); final ArrayList metWriters = setupMetricsWriters( - outputMedium, + segmentWriteOutMedium, mergedMetrics, metricsValueTypes, metricTypeNames, @@ -498,10 +499,10 @@ private void mergeIndexesAndWriteColumns( progress.stopSection(section); } - private LongColumnSerializer setupTimeWriter(OutputMedium outputMedium, IndexSpec indexSpec) throws IOException + private LongColumnSerializer setupTimeWriter(SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec) throws IOException { LongColumnSerializer timeWriter = LongColumnSerializer.create( - outputMedium, + segmentWriteOutMedium, "little_end_time", CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY, indexSpec.getLongEncoding() @@ -512,7 +513,7 @@ private LongColumnSerializer setupTimeWriter(OutputMedium outputMedium, IndexSpe } private ArrayList setupMetricsWriters( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final List mergedMetrics, final Map metricsValueTypes, final Map metricTypeNames, @@ -527,13 +528,13 @@ private ArrayList setupMetricsWriters( GenericColumnSerializer writer; switch (type) { case LONG: - writer = LongColumnSerializer.create(outputMedium, metric, metCompression, longEncoding); + writer = LongColumnSerializer.create(segmentWriteOutMedium, metric, metCompression, longEncoding); break; case FLOAT: - writer = FloatColumnSerializer.create(outputMedium, metric, metCompression); + writer = FloatColumnSerializer.create(segmentWriteOutMedium, metric, metCompression); break; case DOUBLE: - writer = DoubleColumnSerializer.create(outputMedium, metric, metCompression); + writer = DoubleColumnSerializer.create(segmentWriteOutMedium, metric, metCompression); break; case COMPLEX: final String typeName = metricTypeNames.get(metric); @@ -541,7 +542,7 @@ private ArrayList setupMetricsWriters( if (serde == null) { throw new ISE("Unknown type[%s]", typeName); } - writer = serde.getSerializer(outputMedium, metric); + writer = serde.getSerializer(segmentWriteOutMedium, metric); break; default: throw new ISE("Unknown type[%s]", type); @@ -608,10 +609,10 @@ public File persist( final IncrementalIndex index, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { - return persist(index, index.getInterval(), outDir, indexSpec, outputMediumFactory); + return persist(index, index.getInterval(), outDir, indexSpec, segmentWriteOutMediumFactory); } @Override @@ -620,10 +621,10 @@ public File persist( final Interval dataInterval, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { - return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator(), outputMediumFactory); + return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator(), segmentWriteOutMediumFactory); } @Override @@ -633,7 +634,7 @@ public File persist( File outDir, IndexSpec indexSpec, ProgressIndicator progress, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { if (index.isEmpty()) { @@ -671,7 +672,7 @@ public File persist( outDir, indexSpec, progress, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -682,7 +683,7 @@ public File mergeQueryableIndex( final AggregatorFactory[] metricAggs, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { return mergeQueryableIndex( @@ -692,7 +693,7 @@ public File mergeQueryableIndex( outDir, indexSpec, new BaseProgressIndicator(), - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -704,7 +705,7 @@ public File mergeQueryableIndex( File outDir, IndexSpec indexSpec, ProgressIndicator progress, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { return merge( @@ -714,7 +715,7 @@ public File mergeQueryableIndex( outDir, indexSpec, progress, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -737,7 +738,7 @@ private File merge( File outDir, IndexSpec indexSpec, ProgressIndicator progress, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { FileUtils.deleteDirectory(outDir); @@ -843,14 +844,14 @@ public int compare(Rowboat left, Rowboat right) mergedMetrics, rowMergerFn, indexSpec, - outputMediumFactory + segmentWriteOutMediumFactory ); } @Override public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException { - return convert(inDir, outDir, indexSpec, new BaseProgressIndicator(), defaultOutputMediumFactory); + return convert(inDir, outDir, indexSpec, new BaseProgressIndicator(), defaultSegmentWriteOutMediumFactory); } @Override @@ -859,7 +860,7 @@ public File convert( final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress, - final @Nullable OutputMediumFactory outputMediumFactory + final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { try (QueryableIndex index = indexIO.loadIndex(inDir)) { @@ -881,7 +882,7 @@ public Iterable apply(ArrayList> input) } }, indexSpec, - outputMediumFactory + segmentWriteOutMediumFactory ); } } @@ -892,7 +893,7 @@ public File append( AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec, - @Nullable OutputMediumFactory outputMediumFactory + @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) throws IOException { FileUtils.deleteDirectory(outDir); @@ -944,7 +945,7 @@ public Iterable apply( mergedMetrics, rowMergerFn, indexSpec, - outputMediumFactory + segmentWriteOutMediumFactory ); } diff --git a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java index e5f34a6dda06..60dc95cae10d 100644 --- a/processing/src/main/java/io/druid/segment/LongColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/LongColumnSerializer.java @@ -21,7 +21,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.data.CompressionFactory; import io.druid.segment.data.CompressionStrategy; import io.druid.segment.data.LongSupplierSerializer; @@ -36,16 +36,16 @@ public class LongColumnSerializer implements GenericColumnSerializer { public static LongColumnSerializer create( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, CompressionStrategy compression, CompressionFactory.LongEncodingStrategy encoding ) { - return new LongColumnSerializer(outputMedium, filenameBase, IndexIO.BYTE_ORDER, compression, encoding); + return new LongColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression, encoding); } - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ByteOrder byteOrder; private final CompressionStrategy compression; @@ -53,14 +53,14 @@ public static LongColumnSerializer create( private LongSupplierSerializer writer; private LongColumnSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, CompressionStrategy compression, CompressionFactory.LongEncodingStrategy encoding ) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.byteOrder = byteOrder; this.compression = compression; @@ -71,7 +71,7 @@ private LongColumnSerializer( public void open() throws IOException { writer = CompressionFactory.getLongSerializer( - outputMedium, + segmentWriteOutMedium, StringUtils.format("%s.long_column", filenameBase), byteOrder, encoding, diff --git a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java index a936244c4f61..382fd8a8e5e5 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionHandler.java @@ -19,7 +19,7 @@ package io.druid.segment; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.GenericColumn; @@ -53,7 +53,7 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) throws IOException @@ -61,7 +61,7 @@ public DimensionMergerV9 makeMerger( return new LongDimensionMergerV9( dimensionName, indexSpec, - outputMedium, + segmentWriteOutMedium, capabilities, progress ); diff --git a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java index f6891d87ea0d..d3805fcfc42e 100644 --- a/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/LongDimensionMergerV9.java @@ -21,7 +21,7 @@ import com.google.common.base.Throwables; import io.druid.java.util.common.io.Closer; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; @@ -39,13 +39,13 @@ public class LongDimensionMergerV9 implements DimensionMergerV9 protected ProgressIndicator progress; protected final IndexSpec indexSpec; protected ColumnCapabilities capabilities; - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; protected LongColumnSerializer serializer; LongDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -53,7 +53,7 @@ public class LongDimensionMergerV9 implements DimensionMergerV9 this.dimensionName = dimensionName; this.indexSpec = indexSpec; this.capabilities = capabilities; - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.progress = progress; try { @@ -68,7 +68,7 @@ protected void setupEncodedValueWriter() throws IOException { final CompressionStrategy metCompression = indexSpec.getMetricCompression(); final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding(); - this.serializer = LongColumnSerializer.create(outputMedium, dimensionName, metCompression, longEncoding); + this.serializer = LongColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression, longEncoding); serializer.open(); } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java index 85432c7fc7d2..a8adcebad9df 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionHandler.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionHandler.java @@ -21,7 +21,7 @@ import com.google.common.primitives.Ints; import io.druid.data.input.impl.DimensionSchema.MultiValueHandling; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.Column; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.DictionaryEncodedColumn; @@ -192,12 +192,12 @@ public DimensionIndexer makeIndexer() @Override public DimensionMergerV9 makeMerger( IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) { - return new StringDimensionMergerV9(dimensionName, indexSpec, outputMedium, capabilities, progress); + return new StringDimensionMergerV9(dimensionName, indexSpec, segmentWriteOutMedium, capabilities, progress); } } diff --git a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java index 3d3ae2119dcb..f068f6e13915 100644 --- a/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/io/druid/segment/StringDimensionMergerV9.java @@ -32,7 +32,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.logger.Logger; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.column.ColumnCapabilities; import io.druid.segment.column.ColumnDescriptor; import io.druid.segment.column.ValueType; @@ -82,7 +82,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 private boolean convertMissingValues = false; private boolean hasNull = false; private MutableBitmap nullRowsBitmap; - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private int rowCount = 0; private ColumnCapabilities capabilities; private List adapters; @@ -93,7 +93,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9 public StringDimensionMergerV9( String dimensionName, IndexSpec indexSpec, - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, ColumnCapabilities capabilities, ProgressIndicator progress ) @@ -101,7 +101,7 @@ public StringDimensionMergerV9( this.dimensionName = dimensionName; this.indexSpec = indexSpec; this.capabilities = capabilities; - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.progress = progress; nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap(); } @@ -152,7 +152,7 @@ public void writeMergedValueMetadata(List adapters) throws IOE } String dictFilename = StringUtils.format("%s.dim_values", dimensionName); - dictionaryWriter = new GenericIndexedWriter<>(outputMedium, dictFilename, GenericIndexed.STRING_STRATEGY); + dictionaryWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, dictFilename, GenericIndexed.STRING_STRATEGY); firstDictionaryValue = null; dictionarySize = 0; dictionaryWriter.open(); @@ -202,24 +202,24 @@ protected void setupEncodedValueWriter() throws IOException if (capabilities.hasMultipleValues()) { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { encodedValueWriter = CompressedVSizeIndexedV3Writer.create( - outputMedium, + segmentWriteOutMedium, filenameBase, cardinality, compressionStrategy ); } else { - encodedValueWriter = new VSizeIndexedWriter(outputMedium, cardinality); + encodedValueWriter = new VSizeIndexedWriter(segmentWriteOutMedium, cardinality); } } else { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { encodedValueWriter = CompressedVSizeIntsIndexedWriter.create( - outputMedium, + segmentWriteOutMedium, filenameBase, cardinality, compressionStrategy ); } else { - encodedValueWriter = new VSizeIndexedIntsWriter(outputMedium, cardinality); + encodedValueWriter = new VSizeIndexedIntsWriter(segmentWriteOutMedium, cardinality); } } encodedValueWriter.open(); @@ -279,7 +279,7 @@ public void writeIndexes(List segmentRowNumConversions, Closer closer String bmpFilename = StringUtils.format("%s.inverted", dimensionName); bitmapWriter = new GenericIndexedWriter<>( - outputMedium, + segmentWriteOutMedium, bmpFilename, indexSpec.getBitmapSerdeFactory().getObjectStrategy() ); @@ -291,7 +291,7 @@ public void writeIndexes(List segmentRowNumConversions, Closer closer boolean hasSpatial = capabilities.hasSpatialIndexes(); if (hasSpatial) { spatialWriter = new ByteBufferWriter<>( - outputMedium, + segmentWriteOutMedium, new IndexedRTree.ImmutableRTreeObjectStrategy(bitmapFactory) ); spatialWriter.open(); 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 4d8ee30ed420..73115459022e 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutDoubleSupplierSerializer.java @@ -21,7 +21,7 @@ import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedPools; import io.druid.segment.serde.MetaSerdeHelper; @@ -46,21 +46,21 @@ public class BlockLayoutDoubleSupplierSerializer implements DoubleSupplierSerial private ByteBuffer endBuffer; BlockLayoutDoubleSupplierSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, CompressionStrategy compression ) { this.flattener = GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, filenameBase, compression, CompressedPools.BUFFER_SIZE ); this.compression = compression; CompressionStrategy.Compressor compressor = compression.getCompressor(); - Closer closer = outputMedium.getCloser(); + Closer closer = segmentWriteOutMedium.getCloser(); this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder); } 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 f8a844cfb3e9..d0961c801ab7 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutFloatSupplierSerializer.java @@ -21,7 +21,7 @@ import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedPools; import io.druid.segment.serde.MetaSerdeHelper; @@ -45,21 +45,21 @@ public class BlockLayoutFloatSupplierSerializer implements FloatSupplierSerializ private ByteBuffer endBuffer; BlockLayoutFloatSupplierSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, CompressionStrategy compression ) { this.flattener = GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, filenameBase, compression, CompressedPools.BUFFER_SIZE ); this.compression = compression; CompressionStrategy.Compressor compressor = compression.getCompressor(); - Closer closer = outputMedium.getCloser(); + Closer closer = segmentWriteOutMedium.getCloser(); this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder); } 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 e95f1ae77b41..f44a91633d83 100644 --- a/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/BlockLayoutLongSupplierSerializer.java @@ -20,7 +20,7 @@ package io.druid.segment.data; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedPools; import io.druid.segment.serde.MetaSerdeHelper; @@ -47,7 +47,7 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer private ByteBuffer endBuffer = null; BlockLayoutLongSupplierSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, CompressionFactory.LongEncodingWriter writer, @@ -56,11 +56,11 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer { this.sizePer = writer.getBlockSize(CompressedPools.BUFFER_SIZE); int bufferSize = writer.getNumBytes(sizePer); - this.flattener = GenericIndexedWriter.ofCompressedByteBuffers(outputMedium, filenameBase, compression, bufferSize); + 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), outputMedium.getCloser()).order(byteOrder); + endBuffer = compressor.allocateInBuffer(writer.getNumBytes(sizePer), segmentWriteOutMedium.getCloser()).order(byteOrder); writer.setBuffer(endBuffer); numInsertedForNextFlush = sizePer; } 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 5150f6ba0c66..9ecadf185fc7 100644 --- a/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java +++ b/processing/src/main/java/io/druid/segment/data/ByteBufferWriter.java @@ -22,8 +22,8 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.serde.Serializer; import java.io.IOException; @@ -33,22 +33,22 @@ */ public class ByteBufferWriter implements Serializer { - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final ObjectStrategy strategy; - private OutputBytes headerOut = null; - private OutputBytes valueOut = null; + private WriteOutBytes headerOut = null; + private WriteOutBytes valueOut = null; - public ByteBufferWriter(OutputMedium outputMedium, ObjectStrategy strategy) + public ByteBufferWriter(SegmentWriteOutMedium segmentWriteOutMedium, ObjectStrategy strategy) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.strategy = strategy; } public void open() throws IOException { - headerOut = outputMedium.makeOutputBytes(); - valueOut = outputMedium.makeOutputBytes(); + headerOut = segmentWriteOutMedium.makeWriteOutBytes(); + valueOut = segmentWriteOutMedium.makeWriteOutBytes(); } public void write(T objectToWrite) throws IOException diff --git a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java index 18a27714e6c5..1280d929a320 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java @@ -21,7 +21,7 @@ import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.IndexIO; import io.druid.segment.serde.MetaSerdeHelper; @@ -44,13 +44,13 @@ public class CompressedIntsIndexedWriter extends SingleValueIndexedIntsWriter .writeByte(x -> x.compression.getId()); public static CompressedIntsIndexedWriter create( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final CompressionStrategy compression ) { return new CompressedIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, filenameBase, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, IndexIO.BYTE_ORDER, @@ -65,7 +65,7 @@ public static CompressedIntsIndexedWriter create( private int numInserted; CompressedIntsIndexedWriter( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final int chunkFactor, final ByteOrder byteOrder, @@ -73,12 +73,12 @@ public static CompressedIntsIndexedWriter create( ) { this( - outputMedium, + segmentWriteOutMedium, chunkFactor, byteOrder, compression, GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, filenameBase, compression, chunkFactor * Integer.BYTES @@ -87,7 +87,7 @@ public static CompressedIntsIndexedWriter create( } CompressedIntsIndexedWriter( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final int chunkFactor, final ByteOrder byteOrder, final CompressionStrategy compression, @@ -98,7 +98,7 @@ public static CompressedIntsIndexedWriter create( this.compression = compression; this.flattener = flattener; CompressionStrategy.Compressor compressor = compression.getCompressor(); - Closer closer = outputMedium.getCloser(); + Closer closer = segmentWriteOutMedium.getCloser(); this.endBuffer = compressor.allocateInBuffer(chunkFactor * Integer.BYTES, closer).order(byteOrder); this.numInserted = 0; } 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 0607b777abc5..b5b2574f7fa4 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIndexedV3Writer.java @@ -22,7 +22,7 @@ import io.druid.io.Channels; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedVSizeIndexedV3Supplier; import io.druid.segment.IndexIO; import it.unimi.dsi.fastutil.ints.IntList; @@ -40,7 +40,7 @@ public class CompressedVSizeIndexedV3Writer extends MultiValueIndexedIntsWriter private static final byte VERSION = CompressedVSizeIndexedV3Supplier.VERSION; public static CompressedVSizeIndexedV3Writer create( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final int maxValue, final CompressionStrategy compression @@ -48,14 +48,14 @@ public static CompressedVSizeIndexedV3Writer create( { return new CompressedVSizeIndexedV3Writer( new CompressedIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, filenameBase, CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, IndexIO.BYTE_ORDER, compression ), new CompressedVSizeIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, filenameBase, maxValue, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), diff --git a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java index 3a1ae711e001..68176333387e 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriter.java @@ -22,7 +22,7 @@ import com.google.common.primitives.Ints; import io.druid.common.utils.ByteUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.IndexIO; import io.druid.segment.serde.MetaSerdeHelper; @@ -46,14 +46,14 @@ public class CompressedVSizeIntsIndexedWriter extends SingleValueIndexedIntsWrit .writeByte(x -> x.compression.getId()); public static CompressedVSizeIntsIndexedWriter create( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final int maxValue, final CompressionStrategy compression ) { return new CompressedVSizeIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, filenameBase, maxValue, CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForValue(maxValue), @@ -73,7 +73,7 @@ public static CompressedVSizeIntsIndexedWriter create( private int numInserted; CompressedVSizeIntsIndexedWriter( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final int maxValue, final int chunkFactor, @@ -82,13 +82,13 @@ public static CompressedVSizeIntsIndexedWriter create( ) { this( - outputMedium, + segmentWriteOutMedium, maxValue, chunkFactor, byteOrder, compression, GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, filenameBase, compression, sizePer(maxValue, chunkFactor) @@ -97,7 +97,7 @@ public static CompressedVSizeIntsIndexedWriter create( } CompressedVSizeIntsIndexedWriter( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final int maxValue, final int chunkFactor, final ByteOrder byteOrder, @@ -113,7 +113,7 @@ public static CompressedVSizeIntsIndexedWriter create( this.flattener = flattener; this.intBuffer = ByteBuffer.allocate(Ints.BYTES).order(byteOrder); CompressionStrategy.Compressor compressor = compression.getCompressor(); - this.endBuffer = compressor.allocateInBuffer(chunkBytes, outputMedium.getCloser()).order(byteOrder); + this.endBuffer = compressor.allocateInBuffer(chunkBytes, segmentWriteOutMedium.getCloser()).order(byteOrder); this.numInserted = 0; } 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 f2544e6ffd7d..a9b73ca1bb55 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java @@ -25,8 +25,8 @@ import com.google.common.collect.Maps; import io.druid.java.util.common.IAE; import io.druid.java.util.common.StringUtils; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; @@ -217,7 +217,7 @@ public static LongEncodingFormat forId(byte id) /** * This writer output encoded values to the given ByteBuffer or OutputStream. {@link #setBuffer(ByteBuffer)} or - * {@link #setOutputStream(OutputBytes)} 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 @@ -228,7 +228,7 @@ public interface LongEncodingWriter */ void setBuffer(ByteBuffer buffer); - void setOutputStream(OutputBytes output); + void setOutputStream(WriteOutBytes output); void write(long value) throws IOException; @@ -312,7 +312,7 @@ public static Supplier getLongSupplier( } public static LongSupplierSerializer getLongSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder order, LongEncodingStrategy encodingStrategy, @@ -320,13 +320,13 @@ public static LongSupplierSerializer getLongSerializer( ) { if (encodingStrategy == LongEncodingStrategy.AUTO) { - return new IntermediateLongSupplierSerializer(outputMedium, filenameBase, order, compressionStrategy); + return new IntermediateLongSupplierSerializer(segmentWriteOutMedium, filenameBase, order, compressionStrategy); } else if (encodingStrategy == LongEncodingStrategy.LONGS) { if (compressionStrategy == CompressionStrategy.NONE) { - return new EntireLayoutLongSupplierSerializer(outputMedium, new LongsLongEncodingWriter(order)); + return new EntireLayoutLongSupplierSerializer(segmentWriteOutMedium, new LongsLongEncodingWriter(order)); } else { return new BlockLayoutLongSupplierSerializer( - outputMedium, + segmentWriteOutMedium, filenameBase, order, new LongsLongEncodingWriter(order), @@ -356,16 +356,16 @@ public static Supplier getFloatSupplier( } public static FloatSupplierSerializer getFloatSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder order, CompressionStrategy compressionStrategy ) { if (compressionStrategy == CompressionStrategy.NONE) { - return new EntireLayoutFloatSupplierSerializer(outputMedium, order); + return new EntireLayoutFloatSupplierSerializer(segmentWriteOutMedium, order); } else { - return new BlockLayoutFloatSupplierSerializer(outputMedium, filenameBase, order, compressionStrategy); + return new BlockLayoutFloatSupplierSerializer(segmentWriteOutMedium, filenameBase, order, compressionStrategy); } } @@ -387,16 +387,16 @@ public static Supplier getDoubleSupplier( } public static DoubleSupplierSerializer getDoubleSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder byteOrder, CompressionStrategy compression ) { if (compression == CompressionStrategy.NONE) { - return new EntireLayoutDoubleSupplierSerializer(outputMedium, byteOrder); + return new EntireLayoutDoubleSupplierSerializer(segmentWriteOutMedium, byteOrder); } else { - return new BlockLayoutDoubleSupplierSerializer(outputMedium, filenameBase, byteOrder, compression); + return new BlockLayoutDoubleSupplierSerializer(segmentWriteOutMedium, filenameBase, byteOrder, compression); } } } 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 03e43857fb64..7cefc0589fa0 100644 --- a/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java +++ b/processing/src/main/java/io/druid/segment/data/DeltaLongEncodingWriter.java @@ -21,7 +21,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; -import io.druid.output.OutputBytes; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; import java.nio.ByteBuffer; @@ -46,7 +46,7 @@ public void setBuffer(ByteBuffer buffer) } @Override - public void setOutputStream(OutputBytes output) + public void setOutputStream(WriteOutBytes output) { serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); } 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 c0a428a13ff4..6b49acd4e2df 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutDoubleSupplierSerializer.java @@ -20,8 +20,8 @@ package io.druid.segment.data; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; @@ -38,15 +38,15 @@ public class EntireLayoutDoubleSupplierSerializer implements DoubleSupplierSeria .writeInt(x -> 0) .writeByte(x -> CompressionStrategy.NONE.getId()); - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final ByteBuffer orderBuffer; - private OutputBytes valuesOut; + private WriteOutBytes valuesOut; private int numInserted = 0; - public EntireLayoutDoubleSupplierSerializer(OutputMedium outputMedium, ByteOrder order) + public EntireLayoutDoubleSupplierSerializer(SegmentWriteOutMedium segmentWriteOutMedium, ByteOrder order) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.orderBuffer = ByteBuffer.allocate(Double.BYTES); orderBuffer.order(order); } @@ -54,7 +54,7 @@ public EntireLayoutDoubleSupplierSerializer(OutputMedium outputMedium, ByteOrder @Override public void open() throws IOException { - valuesOut = outputMedium.makeOutputBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override 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 cc39f14ee27b..8595d0121d7a 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutFloatSupplierSerializer.java @@ -20,8 +20,8 @@ package io.druid.segment.data; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; @@ -37,21 +37,21 @@ public class EntireLayoutFloatSupplierSerializer implements FloatSupplierSeriali .writeByte(x -> CompressionStrategy.NONE.getId()); private final boolean isLittleEndian; - private final OutputMedium outputMedium; - private OutputBytes valuesOut; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private WriteOutBytes valuesOut; private int numInserted = 0; - EntireLayoutFloatSupplierSerializer(OutputMedium outputMedium, ByteOrder order) + EntireLayoutFloatSupplierSerializer(SegmentWriteOutMedium segmentWriteOutMedium, ByteOrder order) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; isLittleEndian = order.equals(ByteOrder.LITTLE_ENDIAN); } @Override public void open() throws IOException { - valuesOut = outputMedium.makeOutputBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override @@ -64,7 +64,7 @@ public int size() public void add(float value) throws IOException { int valueBits = Float.floatToRawIntBits(value); - // OutputBytes are always big-endian, so need to reverse bytes + // WriteOutBytes are always big-endian, so need to reverse bytes if (isLittleEndian) { valueBits = Integer.reverseBytes(valueBits); } 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 4566178c4a8c..a1a31902a717 100644 --- a/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/EntireLayoutLongSupplierSerializer.java @@ -20,8 +20,8 @@ package io.druid.segment.data; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; @@ -36,21 +36,21 @@ public class EntireLayoutLongSupplierSerializer implements LongSupplierSerialize .writeSomething(CompressionFactory.longEncodingWriter(x -> x.writer, x -> CompressionStrategy.NONE)); private final CompressionFactory.LongEncodingWriter writer; - private final OutputMedium outputMedium; - private OutputBytes valuesOut; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private WriteOutBytes valuesOut; private int numInserted = 0; - EntireLayoutLongSupplierSerializer(OutputMedium outputMedium, CompressionFactory.LongEncodingWriter writer) + EntireLayoutLongSupplierSerializer(SegmentWriteOutMedium segmentWriteOutMedium, CompressionFactory.LongEncodingWriter writer) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.writer = writer; } @Override public void open() throws IOException { - valuesOut = outputMedium.makeOutputBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); writer.setOutputStream(valuesOut); } 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 46eaa8369941..2bbfa9ef6568 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexed.java @@ -31,7 +31,7 @@ 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.output.HeapByteBufferOutputBytes; +import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.serde.MetaSerdeHelper; import io.druid.segment.serde.Serializer; @@ -485,8 +485,8 @@ private static GenericIndexed fromIterableVersionOne( int count = 0; - HeapByteBufferOutputBytes headerOut = new HeapByteBufferOutputBytes(); - HeapByteBufferOutputBytes valuesOut = new HeapByteBufferOutputBytes(); + HeapByteBufferWriteOutBytes headerOut = new HeapByteBufferWriteOutBytes(); + HeapByteBufferWriteOutBytes valuesOut = new HeapByteBufferWriteOutBytes(); try { T prevVal = null; do { 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 8a0482a81611..6c6afd39a329 100644 --- a/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/GenericIndexedWriter.java @@ -27,8 +27,8 @@ 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.output.OutputBytes; -import io.druid.output.OutputMedium; +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; @@ -71,16 +71,16 @@ public class GenericIndexedWriter implements Serializer static GenericIndexedWriter ofCompressedByteBuffers( - final OutputMedium outputMedium, + final SegmentWriteOutMedium segmentWriteOutMedium, final String filenameBase, final CompressionStrategy compressionStrategy, final int bufferSize ) { GenericIndexedWriter writer = new GenericIndexedWriter<>( - outputMedium, + segmentWriteOutMedium, filenameBase, - compressedByteBuffersWriteObjectStrategy(compressionStrategy, bufferSize, outputMedium.getCloser()) + compressedByteBuffersWriteObjectStrategy(compressionStrategy, bufferSize, segmentWriteOutMedium.getCloser()) ); writer.objectsSorted = false; return writer; @@ -116,7 +116,7 @@ public byte[] toBytes(ByteBuffer val) } @Override - public void writeTo(ByteBuffer val, OutputBytes out) throws IOException + public void writeTo(ByteBuffer val, WriteOutBytes out) throws IOException { compressedDataBuffer.clear(); int valPos = val.position(); @@ -132,34 +132,34 @@ public int compare(ByteBuffer o1, ByteBuffer o2) }; } - private final OutputMedium outputMedium; + 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 OutputBytes headerOut = null; - private OutputBytes valuesOut = null; + private WriteOutBytes headerOut = null; + private WriteOutBytes valuesOut = null; private int numWritten = 0; private boolean requireMultipleFiles = false; private LongList headerOutLong; private final ByteBuffer getOffsetBuffer = ByteBuffer.allocate(Integer.BYTES); - public GenericIndexedWriter(OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy) + public GenericIndexedWriter(SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy) { - this(outputMedium, filenameBase, strategy, Integer.MAX_VALUE & ~PAGE_SIZE); + this(segmentWriteOutMedium, filenameBase, strategy, Integer.MAX_VALUE & ~PAGE_SIZE); } public GenericIndexedWriter( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy, int fileSizeLimit ) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.strategy = strategy; this.fileSizeLimit = fileSizeLimit; @@ -198,8 +198,8 @@ private static void writeBytesIntoSmooshedChannel( public void open() throws IOException { - headerOut = outputMedium.makeOutputBytes(); - valuesOut = outputMedium.makeOutputBytes(); + headerOut = segmentWriteOutMedium.makeWriteOutBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } public void write(T objectToWrite) throws IOException diff --git a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java index fbf55e033646..8c2be60c1509 100644 --- a/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java +++ b/processing/src/main/java/io/druid/segment/data/IntermediateLongSupplierSerializer.java @@ -21,7 +21,7 @@ import com.google.common.math.LongMath; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import it.unimi.dsi.fastutil.longs.Long2IntMap; import it.unimi.dsi.fastutil.longs.Long2IntOpenHashMap; import it.unimi.dsi.fastutil.longs.LongArrayList; @@ -36,7 +36,7 @@ */ public class IntermediateLongSupplierSerializer implements LongSupplierSerializer { - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ByteOrder order; private final CompressionStrategy compression; @@ -53,13 +53,13 @@ public class IntermediateLongSupplierSerializer implements LongSupplierSerialize private LongSupplierSerializer delegate; IntermediateLongSupplierSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ByteOrder order, CompressionStrategy compression ) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.order = order; this.compression = compression; @@ -121,9 +121,9 @@ private void makeDelegate() throws IOException } if (compression == CompressionStrategy.NONE) { - delegate = new EntireLayoutLongSupplierSerializer(outputMedium, writer); + delegate = new EntireLayoutLongSupplierSerializer(segmentWriteOutMedium, writer); } else { - delegate = new BlockLayoutLongSupplierSerializer(outputMedium, filenameBase, order, writer, compression); + delegate = new BlockLayoutLongSupplierSerializer(segmentWriteOutMedium, filenameBase, order, writer, compression); } delegate.open(); 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 2a233d765a66..baa01879ae93 100644 --- a/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java +++ b/processing/src/main/java/io/druid/segment/data/LongsLongEncodingWriter.java @@ -20,7 +20,7 @@ package io.druid.segment.data; import com.google.common.primitives.Longs; -import io.druid.output.OutputBytes; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; import java.io.OutputStream; @@ -53,7 +53,7 @@ public void setBuffer(ByteBuffer buffer) } @Override - public void setOutputStream(OutputBytes output) + public void setOutputStream(WriteOutBytes output) { outBuffer = null; outStream = output; 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 8ae81279b2e0..e5957a7ee947 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,7 @@ package io.druid.segment.data; import io.druid.guice.annotations.ExtensionPoint; -import io.druid.output.OutputBytes; +import io.druid.segment.writeout.WriteOutBytes; import java.io.IOException; import java.nio.ByteBuffer; @@ -47,7 +47,7 @@ public interface ObjectStrategy extends Comparator T fromByteBuffer(ByteBuffer buffer, int numBytes); byte[] toBytes(T val); - default void writeTo(T val, OutputBytes out) throws IOException + default void writeTo(T val, WriteOutBytes out) throws IOException { out.write(toBytes(val)); } 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 ff90480fb209..36382a2b13b5 100644 --- a/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java +++ b/processing/src/main/java/io/druid/segment/data/TableLongEncodingWriter.java @@ -22,7 +22,7 @@ import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import io.druid.java.util.common.IAE; -import io.druid.output.OutputBytes; +import io.druid.segment.writeout.WriteOutBytes; import it.unimi.dsi.fastutil.longs.Long2IntMap; import it.unimi.dsi.fastutil.longs.LongList; @@ -54,7 +54,7 @@ public void setBuffer(ByteBuffer buffer) } @Override - public void setOutputStream(OutputBytes output) + public void setOutputStream(WriteOutBytes output) { serializer = VSizeLongSerde.getSerializer(bitsPerValue, output); } 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 0560fec7b814..84db1835c267 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexed.java @@ -25,7 +25,7 @@ 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.output.HeapByteBufferOutputBytes; +import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.serde.MetaSerdeHelper; @@ -65,8 +65,8 @@ public static VSizeIndexed fromIterable(Iterable objectsIterab ++count; } - HeapByteBufferOutputBytes headerBytes = new HeapByteBufferOutputBytes(); - HeapByteBufferOutputBytes valueBytes = new HeapByteBufferOutputBytes(); + HeapByteBufferWriteOutBytes headerBytes = new HeapByteBufferWriteOutBytes(); + HeapByteBufferWriteOutBytes valueBytes = new HeapByteBufferWriteOutBytes(); int offset = 0; try { headerBytes.writeInt(count); 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 b79149aa816a..285328bd84d4 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -24,7 +24,7 @@ import io.druid.io.Channels; import io.druid.java.util.common.IAE; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.HeapByteBufferOutputBytes; +import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; import io.druid.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.serde.MetaSerdeHelper; import it.unimi.dsi.fastutil.ints.IntArrayList; @@ -139,7 +139,7 @@ public int getNumBytesNoPadding() return buffer.remaining() - (Ints.BYTES - numBytes); } - public void writeBytesNoPaddingTo(HeapByteBufferOutputBytes out) + public void writeBytesNoPaddingTo(HeapByteBufferWriteOutBytes out) { ByteBuffer toWrite = buffer.slice(); toWrite.limit(toWrite.limit() - (Ints.BYTES - numBytes)); 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 33d75b05e327..edbaf2f4f6ca 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedIntsWriter.java @@ -22,8 +22,8 @@ import com.google.common.primitives.Ints; import io.druid.common.utils.ByteUtils; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.serde.MetaSerdeHelper; import java.io.IOException; @@ -42,23 +42,23 @@ public class VSizeIndexedIntsWriter extends SingleValueIndexedIntsWriter .writeByte(x -> ByteUtils.checkedCast(x.numBytes)) .writeInt(x -> Ints.checkedCast(x.valuesOut.size())); - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final int numBytes; private final ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES); - private OutputBytes valuesOut = null; + private WriteOutBytes valuesOut = null; private boolean bufPaddingWritten = false; - public VSizeIndexedIntsWriter(final OutputMedium outputMedium, final int maxValue) + public VSizeIndexedIntsWriter(final SegmentWriteOutMedium segmentWriteOutMedium, final int maxValue) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.numBytes = VSizeIndexedInts.getNumBytesForMax(maxValue); } @Override public void open() throws IOException { - valuesOut = outputMedium.makeOutputBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override 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 31ff83fe8e98..7f4f1d4e1f2e 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedWriter.java @@ -22,8 +22,8 @@ import com.google.common.base.Preconditions; import com.google.common.primitives.Ints; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +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; @@ -47,14 +47,14 @@ private enum WriteInt { ONE_BYTE { @Override - void write(OutputBytes out, int v) throws IOException + void write(WriteOutBytes out, int v) throws IOException { out.write(v); } }, TWO_BYTES { @Override - void write(OutputBytes out, int v) throws IOException + void write(WriteOutBytes out, int v) throws IOException { out.write(v >> 8); out.write(v); @@ -62,7 +62,7 @@ void write(OutputBytes out, int v) throws IOException }, THREE_BYTES { @Override - void write(OutputBytes out, int v) throws IOException + void write(WriteOutBytes out, int v) throws IOException { out.write(v >> 16); out.write(v >> 8); @@ -71,27 +71,27 @@ void write(OutputBytes out, int v) throws IOException }, FOUR_BYTES { @Override - void write(OutputBytes out, int v) throws IOException + void write(WriteOutBytes out, int v) throws IOException { out.writeInt(v); } }; - abstract void write(OutputBytes out, int v) throws IOException; + abstract void write(WriteOutBytes out, int v) throws IOException; } private final int maxId; private final WriteInt writeInt; - private final OutputMedium outputMedium; - private OutputBytes headerOut = null; - private OutputBytes valuesOut = null; + private final SegmentWriteOutMedium segmentWriteOutMedium; + private WriteOutBytes headerOut = null; + private WriteOutBytes valuesOut = null; private int numWritten = 0; private boolean numBytesForMaxWritten = false; - public VSizeIndexedWriter(OutputMedium outputMedium, int maxId) + public VSizeIndexedWriter(SegmentWriteOutMedium segmentWriteOutMedium, int maxId) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.maxId = maxId; this.writeInt = WriteInt.values()[VSizeIndexedInts.getNumBytesForMax(maxId) - 1]; } @@ -99,8 +99,8 @@ public VSizeIndexedWriter(OutputMedium outputMedium, int maxId) @Override public void open() throws IOException { - headerOut = outputMedium.makeOutputBytes(); - valuesOut = outputMedium.makeOutputBytes(); + headerOut = segmentWriteOutMedium.makeWriteOutBytes(); + valuesOut = segmentWriteOutMedium.makeWriteOutBytes(); } @Override 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 91aff3fc1c7e..97798eaeddcf 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexColumnSerializer.java @@ -22,7 +22,7 @@ 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.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.ObjectStrategy; @@ -33,19 +33,19 @@ public class ComplexColumnSerializer implements GenericColumnSerializer { @PublicApi - public static ComplexColumnSerializer create(OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy) + public static ComplexColumnSerializer create(SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy) { - return new ComplexColumnSerializer(outputMedium, filenameBase, strategy); + return new ComplexColumnSerializer(segmentWriteOutMedium, filenameBase, strategy); } - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ObjectStrategy strategy; private GenericIndexedWriter writer; - private ComplexColumnSerializer(OutputMedium outputMedium, String filenameBase, ObjectStrategy strategy) + private ComplexColumnSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.strategy = strategy; } @@ -54,7 +54,7 @@ private ComplexColumnSerializer(OutputMedium outputMedium, String filenameBase, @Override public void open() throws IOException { - writer = new GenericIndexedWriter(outputMedium, StringUtils.format("%s.complex_column", filenameBase), strategy); + writer = new GenericIndexedWriter(segmentWriteOutMedium, StringUtils.format("%s.complex_column", filenameBase), strategy); writer.open(); } 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 ddc03deba610..dc3c6647f3a1 100644 --- a/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/ComplexMetricSerde.java @@ -21,7 +21,7 @@ import com.google.common.base.Function; import io.druid.guice.annotations.ExtensionPoint; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.ObjectStrategy; @@ -110,8 +110,8 @@ public Object fromBytes(byte[] data, int start, int numBytes) * * @return an instance of GenericColumnSerializer used for serialization. */ - public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return ComplexColumnSerializer.create(outputMedium, column, this.getObjectStrategy()); + return ComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); } } 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 e8e762389da0..6486f7eeaa06 100644 --- a/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java +++ b/processing/src/main/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializer.java @@ -22,7 +22,7 @@ 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.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.data.GenericIndexedWriter; import io.druid.segment.data.ObjectStrategy; @@ -34,47 +34,47 @@ public class LargeColumnSupportedComplexColumnSerializer implements GenericColum { @PublicApi public static LargeColumnSupportedComplexColumnSerializer create( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy ) { - return new LargeColumnSupportedComplexColumnSerializer(outputMedium, filenameBase, strategy); + return new LargeColumnSupportedComplexColumnSerializer(segmentWriteOutMedium, filenameBase, strategy); } public static LargeColumnSupportedComplexColumnSerializer createWithColumnSize( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy, int columnSize ) { - return new LargeColumnSupportedComplexColumnSerializer(outputMedium, filenameBase, strategy, columnSize); + return new LargeColumnSupportedComplexColumnSerializer(segmentWriteOutMedium, filenameBase, strategy, columnSize); } - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; private final String filenameBase; private final ObjectStrategy strategy; private final int columnSize; private GenericIndexedWriter writer; private LargeColumnSupportedComplexColumnSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy ) { - this(outputMedium, filenameBase, strategy, Integer.MAX_VALUE); + this(segmentWriteOutMedium, filenameBase, strategy, Integer.MAX_VALUE); } private LargeColumnSupportedComplexColumnSerializer( - OutputMedium outputMedium, + SegmentWriteOutMedium segmentWriteOutMedium, String filenameBase, ObjectStrategy strategy, int columnSize ) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; this.filenameBase = filenameBase; this.strategy = strategy; this.columnSize = columnSize; @@ -85,7 +85,7 @@ private LargeColumnSupportedComplexColumnSerializer( public void open() throws IOException { writer = new GenericIndexedWriter( - outputMedium, + segmentWriteOutMedium, StringUtils.format("%s.complex_column", filenameBase), strategy, columnSize diff --git a/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java b/processing/src/main/java/io/druid/segment/writeout/ByteBufferWriteOutBytes.java similarity index 95% rename from processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java rename to processing/src/main/java/io/druid/segment/writeout/ByteBufferWriteOutBytes.java index 985bb40bf581..263c74679daa 100644 --- a/processing/src/main/java/io/druid/output/ByteBufferOutputBytes.java +++ b/processing/src/main/java/io/druid/segment/writeout/ByteBufferWriteOutBytes.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.google.common.base.Preconditions; import com.google.common.io.ByteSource; @@ -35,7 +35,7 @@ import java.util.function.Function; import java.util.stream.Collectors; -public abstract class ByteBufferOutputBytes extends OutputBytes +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. @@ -48,7 +48,7 @@ public abstract class ByteBufferOutputBytes extends OutputBytes long size; long capacity; - ByteBufferOutputBytes() + ByteBufferWriteOutBytes() { size = 0; headBufferIndex = 0; @@ -183,11 +183,11 @@ public void writeTo(WritableByteChannel channel) throws IOException } /** - * Takes all bytes that are written to this OutputBytes so far and writes them into the given ByteBuffer. This method - * changes the position of the out buffer by the {@link #size()} of this OutputBytes. + * 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 OutputBytes + * smaller than the size of this WriteOutBytes */ public void writeTo(ByteBuffer out) { diff --git a/processing/src/main/java/io/druid/output/DirectByteBufferOutputBytes.java b/processing/src/main/java/io/druid/segment/writeout/DirectByteBufferWriteOutBytes.java similarity index 92% rename from processing/src/main/java/io/druid/output/DirectByteBufferOutputBytes.java rename to processing/src/main/java/io/druid/segment/writeout/DirectByteBufferWriteOutBytes.java index 29c30da51c1b..172c99afb78f 100644 --- a/processing/src/main/java/io/druid/output/DirectByteBufferOutputBytes.java +++ b/processing/src/main/java/io/druid/segment/writeout/DirectByteBufferWriteOutBytes.java @@ -17,13 +17,13 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import io.druid.java.util.common.ByteBufferUtils; import java.nio.ByteBuffer; -final class DirectByteBufferOutputBytes extends ByteBufferOutputBytes +final class DirectByteBufferWriteOutBytes extends ByteBufferWriteOutBytes { private boolean open = true; diff --git a/processing/src/main/java/io/druid/output/FileOutputBytes.java b/processing/src/main/java/io/druid/segment/writeout/FileWriteOutBytes.java similarity index 96% rename from processing/src/main/java/io/druid/output/FileOutputBytes.java rename to processing/src/main/java/io/druid/segment/writeout/FileWriteOutBytes.java index b73b5b1913e3..9844afd94970 100644 --- a/processing/src/main/java/io/druid/output/FileOutputBytes.java +++ b/processing/src/main/java/io/druid/segment/writeout/FileWriteOutBytes.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.google.common.io.ByteStreams; import io.druid.io.Channels; @@ -32,7 +32,7 @@ import java.nio.channels.FileChannel; import java.nio.channels.WritableByteChannel; -final class FileOutputBytes extends OutputBytes +final class FileWriteOutBytes extends WriteOutBytes { private final File file; private final FileChannel ch; @@ -40,7 +40,7 @@ final class FileOutputBytes extends OutputBytes /** Purposely big-endian, for {@link #writeInt(int)} implementation */ private final ByteBuffer buffer = ByteBuffer.allocate(4096); // 4K page sized buffer - FileOutputBytes(File file, FileChannel ch) + FileWriteOutBytes(File file, FileChannel ch) { this.file = file; this.ch = ch; diff --git a/processing/src/main/java/io/druid/output/HeapByteBufferOutputBytes.java b/processing/src/main/java/io/druid/segment/writeout/HeapByteBufferWriteOutBytes.java similarity index 89% rename from processing/src/main/java/io/druid/output/HeapByteBufferOutputBytes.java rename to processing/src/main/java/io/druid/segment/writeout/HeapByteBufferWriteOutBytes.java index 163f1cfd14b5..4823d49375d1 100644 --- a/processing/src/main/java/io/druid/output/HeapByteBufferOutputBytes.java +++ b/processing/src/main/java/io/druid/segment/writeout/HeapByteBufferWriteOutBytes.java @@ -17,11 +17,11 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import java.nio.ByteBuffer; -public final class HeapByteBufferOutputBytes extends ByteBufferOutputBytes +public final class HeapByteBufferWriteOutBytes extends ByteBufferWriteOutBytes { @Override protected ByteBuffer allocateBuffer() diff --git a/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMedium.java b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMedium.java similarity index 77% rename from processing/src/main/java/io/druid/output/OffHeapMemoryOutputMedium.java rename to processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMedium.java index 1583e2bf0ab0..4a8974d129e5 100644 --- a/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMedium.java +++ b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMedium.java @@ -17,22 +17,22 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import io.druid.java.util.common.io.Closer; import java.io.IOException; -public final class OffHeapMemoryOutputMedium implements OutputMedium +public final class OffHeapMemorySegmentWriteOutMedium implements SegmentWriteOutMedium { private final Closer closer = Closer.create(); @Override - public OutputBytes makeOutputBytes() + public WriteOutBytes makeWriteOutBytes() { - DirectByteBufferOutputBytes outputBytes = new DirectByteBufferOutputBytes(); - closer.register(outputBytes::free); - return outputBytes; + DirectByteBufferWriteOutBytes writeOutBytes = new DirectByteBufferWriteOutBytes(); + closer.register(writeOutBytes::free); + return writeOutBytes; } @Override diff --git a/processing/src/main/java/io/druid/output/TmpFileOutputMediumFactory.java b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMediumFactory.java similarity index 64% rename from processing/src/main/java/io/druid/output/TmpFileOutputMediumFactory.java rename to processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMediumFactory.java index ac8b434ab37e..d45c5b7908d6 100644 --- a/processing/src/main/java/io/druid/output/TmpFileOutputMediumFactory.java +++ b/processing/src/main/java/io/druid/segment/writeout/OffHeapMemorySegmentWriteOutMediumFactory.java @@ -17,27 +17,29 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.fasterxml.jackson.annotation.JsonCreator; import java.io.File; import java.io.IOException; -public final class TmpFileOutputMediumFactory implements OutputMediumFactory +public class OffHeapMemorySegmentWriteOutMediumFactory implements SegmentWriteOutMediumFactory { - private static final TmpFileOutputMediumFactory INSTANCE = new TmpFileOutputMediumFactory(); + private static final OffHeapMemorySegmentWriteOutMediumFactory INSTANCE = + new OffHeapMemorySegmentWriteOutMediumFactory(); @JsonCreator - public static TmpFileOutputMediumFactory instance() + public static OffHeapMemorySegmentWriteOutMediumFactory instance() { return INSTANCE; } - private TmpFileOutputMediumFactory() {} + private OffHeapMemorySegmentWriteOutMediumFactory() {} + @Override - public OutputMedium makeOutputMedium(File outDir) throws IOException + public SegmentWriteOutMedium makeSegmentWriteOutMedium(File outDir) throws IOException { - return new TmpFileOutputMedium(outDir); + return new OffHeapMemorySegmentWriteOutMedium(); } } diff --git a/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java b/processing/src/main/java/io/druid/segment/writeout/OnHeapMemorySegmentWriteOutMedium.java similarity index 83% rename from processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java rename to processing/src/main/java/io/druid/segment/writeout/OnHeapMemorySegmentWriteOutMedium.java index ebd36973b491..ab83db62a6ff 100644 --- a/processing/src/main/java/io/druid/output/OnHeapMemoryOutputMedium.java +++ b/processing/src/main/java/io/druid/segment/writeout/OnHeapMemorySegmentWriteOutMedium.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.google.common.annotations.VisibleForTesting; import io.druid.java.util.common.io.Closer; @@ -25,14 +25,14 @@ import java.io.IOException; @VisibleForTesting -public final class OnHeapMemoryOutputMedium implements OutputMedium +public final class OnHeapMemorySegmentWriteOutMedium implements SegmentWriteOutMedium { private final Closer closer = Closer.create(); @Override - public OutputBytes makeOutputBytes() throws IOException + public WriteOutBytes makeWriteOutBytes() throws IOException { - return new HeapByteBufferOutputBytes(); + return new HeapByteBufferWriteOutBytes(); } @Override 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 000000000000..4d16b7695bbc --- /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/output/OutputMediumFactory.java b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumFactory.java similarity index 56% rename from processing/src/main/java/io/druid/output/OutputMediumFactory.java rename to processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumFactory.java index 4e7da59e512c..29c606ba23a0 100644 --- a/processing/src/main/java/io/druid/output/OutputMediumFactory.java +++ b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumFactory.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; @@ -27,22 +27,25 @@ import java.io.IOException; import java.util.Set; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, defaultImpl = TmpFileOutputMediumFactory.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, defaultImpl = TmpFileSegmentWriteOutMediumFactory.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "tmpFile", value = TmpFileOutputMediumFactory.class), - @JsonSubTypes.Type(name = "offHeapMemory", value = OffHeapMemoryOutputMediumFactory.class), + @JsonSubTypes.Type(name = "tmpFile", value = TmpFileSegmentWriteOutMediumFactory.class), + @JsonSubTypes.Type(name = "offHeapMemory", value = OffHeapMemorySegmentWriteOutMediumFactory.class), }) -public interface OutputMediumFactory +public interface SegmentWriteOutMediumFactory { - static Set builtInFactories() + static Set builtInFactories() { - return ImmutableSet.of(TmpFileOutputMediumFactory.instance(), OffHeapMemoryOutputMediumFactory.instance()); + return ImmutableSet.of( + TmpFileSegmentWriteOutMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance() + ); } /** - * Creates a new OutputMedium. If this type of OutputMedium 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. + * 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. */ - OutputMedium makeOutputMedium(File outDir) throws IOException; + SegmentWriteOutMedium makeSegmentWriteOutMedium(File outDir) throws IOException; } diff --git a/processing/src/main/java/io/druid/output/OutputMediumModule.java b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumModule.java similarity index 79% rename from processing/src/main/java/io/druid/output/OutputMediumModule.java rename to processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumModule.java index 68744cde0edd..3f7a6fed2043 100644 --- a/processing/src/main/java/io/druid/output/OutputMediumModule.java +++ b/processing/src/main/java/io/druid/segment/writeout/SegmentWriteOutMediumModule.java @@ -17,17 +17,21 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.google.inject.Binder; import com.google.inject.Module; import io.druid.guice.JsonConfigProvider; -public class OutputMediumModule implements Module +public class SegmentWriteOutMediumModule implements Module { @Override public void configure(Binder binder) { - JsonConfigProvider.bind(binder, "druid.peon.defaultOutputMediumFactory", OutputMediumFactory.class); + JsonConfigProvider.bind( + binder, + "druid.peon.defaultSegmentWriteOutMediumFactory", + SegmentWriteOutMediumFactory.class + ); } } diff --git a/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMedium.java similarity index 86% rename from processing/src/main/java/io/druid/output/TmpFileOutputMedium.java rename to processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMedium.java index 6cf50a6e2cd7..b3ee365e7324 100644 --- a/processing/src/main/java/io/druid/output/TmpFileOutputMedium.java +++ b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMedium.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import io.druid.java.util.common.io.Closer; import org.apache.commons.io.FileUtils; @@ -27,12 +27,12 @@ import java.nio.channels.FileChannel; import java.nio.file.StandardOpenOption; -public final class TmpFileOutputMedium implements OutputMedium +public final class TmpFileSegmentWriteOutMedium implements SegmentWriteOutMedium { private final File dir; private final Closer closer = Closer.create(); - TmpFileOutputMedium(File outDir) throws IOException + TmpFileSegmentWriteOutMedium(File outDir) throws IOException { File tmpOutputFilesDir = new File(outDir, "tmpOutputFiles"); FileUtils.forceMkdir(tmpOutputFilesDir); @@ -41,7 +41,7 @@ public final class TmpFileOutputMedium implements OutputMedium } @Override - public OutputBytes makeOutputBytes() throws IOException + public WriteOutBytes makeWriteOutBytes() throws IOException { File file = File.createTempFile("filePeon", null, dir); FileChannel ch = FileChannel.open( @@ -51,7 +51,7 @@ public OutputBytes makeOutputBytes() throws IOException ); closer.register(file::delete); closer.register(ch); - return new FileOutputBytes(file, ch); + return new FileWriteOutBytes(file, ch); } @Override diff --git a/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMediumFactory.java b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMediumFactory.java similarity index 66% rename from processing/src/main/java/io/druid/output/OffHeapMemoryOutputMediumFactory.java rename to processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMediumFactory.java index 49fda6e1b6ee..7fce54f07aeb 100644 --- a/processing/src/main/java/io/druid/output/OffHeapMemoryOutputMediumFactory.java +++ b/processing/src/main/java/io/druid/segment/writeout/TmpFileSegmentWriteOutMediumFactory.java @@ -17,28 +17,28 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.fasterxml.jackson.annotation.JsonCreator; import java.io.File; import java.io.IOException; -public class OffHeapMemoryOutputMediumFactory implements OutputMediumFactory +public final class TmpFileSegmentWriteOutMediumFactory implements SegmentWriteOutMediumFactory { - private static final OffHeapMemoryOutputMediumFactory INSTANCE = new OffHeapMemoryOutputMediumFactory(); + private static final TmpFileSegmentWriteOutMediumFactory INSTANCE = new TmpFileSegmentWriteOutMediumFactory(); @JsonCreator - public static OffHeapMemoryOutputMediumFactory instance() + public static TmpFileSegmentWriteOutMediumFactory instance() { return INSTANCE; } - private OffHeapMemoryOutputMediumFactory() {} + private TmpFileSegmentWriteOutMediumFactory() {} @Override - public OutputMedium makeOutputMedium(File outDir) throws IOException + public SegmentWriteOutMedium makeSegmentWriteOutMedium(File outDir) throws IOException { - return new OffHeapMemoryOutputMedium(); + return new TmpFileSegmentWriteOutMedium(outDir); } } diff --git a/processing/src/main/java/io/druid/output/OutputBytes.java b/processing/src/main/java/io/druid/segment/writeout/WriteOutBytes.java similarity index 75% rename from processing/src/main/java/io/druid/output/OutputBytes.java rename to processing/src/main/java/io/druid/segment/writeout/WriteOutBytes.java index 04a4a63e8352..312341ade448 100644 --- a/processing/src/main/java/io/druid/output/OutputBytes.java +++ b/processing/src/main/java/io/druid/segment/writeout/WriteOutBytes.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import java.io.IOException; import java.io.InputStream; @@ -31,11 +31,11 @@ * #writeTo(WritableByteChannel)} and {@link #asInputStream()} allow to write the sequence somewhere else. {@link * #readFully} allows to access the sequence randomly. * - * OutputBytes is a resource that is managed by {@link OutputMedium}, so it's own {@link #close()} method does nothing. - * However OutputBytes should appear closed, i. e. {@link #isOpen()} returns false, after the parental OutputMedium is - * closed. + * 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 OutputBytes extends OutputStream implements WritableByteChannel +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)}. @@ -43,23 +43,23 @@ public abstract class OutputBytes extends OutputStream implements WritableByteCh public abstract void writeInt(int v) throws IOException; /** - * Returns the number of bytes written to this OutputBytes so far. + * 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 OutputBytes so far and writes them into the given channel. + * 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 OutputBytes so far. The returned + * 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 OutputBytes, at the random position, into the given + * 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 diff --git a/processing/src/test/java/io/druid/query/DoubleStorageTest.java b/processing/src/test/java/io/druid/query/DoubleStorageTest.java index f45d81999c5d..65b2d7c7ad2d 100644 --- a/processing/src/test/java/io/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/io/druid/query/DoubleStorageTest.java @@ -32,7 +32,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.metadata.SegmentMetadataQueryConfig; import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest; @@ -112,8 +112,8 @@ private ScanQuery.ScanQueryBuilder newTestQuery() private static final IndexMergerV9 INDEX_MERGER_V9 = - TestHelper.getTestIndexMergerV9(OffHeapMemoryOutputMediumFactory.instance()); - private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()); + 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"; diff --git a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java index d07c6955ffc9..f3708608ba86 100644 --- a/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/io/druid/query/MultiValuedDimensionTest.java @@ -35,9 +35,9 @@ 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.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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; @@ -89,21 +89,21 @@ public static Collection constructorFeeder() throws IOException { final List constructors = Lists.newArrayList(); for (GroupByQueryConfig config : GroupByQueryRunnerTest.testConfigs()) { - constructors.add(new Object[]{config, TmpFileOutputMediumFactory.instance()}); - constructors.add(new Object[]{config, OffHeapMemoryOutputMediumFactory.instance()}); + constructors.add(new Object[]{config, TmpFileSegmentWriteOutMediumFactory.instance()}); + constructors.add(new Object[]{config, OffHeapMemorySegmentWriteOutMediumFactory.instance()}); } return constructors; } private final AggregationTestHelper helper; - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private IncrementalIndex incrementalIndex; private QueryableIndex queryableIndex; private File persistedSegmentDir; - public MultiValuedDimensionTest(final GroupByQueryConfig config, OutputMediumFactory outputMediumFactory) + public MultiValuedDimensionTest(final GroupByQueryConfig config, SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) throws Exception { helper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( @@ -111,7 +111,7 @@ public MultiValuedDimensionTest(final GroupByQueryConfig config, OutputMediumFac config, null ); - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Before @@ -146,10 +146,10 @@ public void setup() throws Exception } persistedSegmentDir = Files.createTempDir(); - TestHelper.getTestIndexMergerV9(outputMediumFactory) + TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory) .persist(incrementalIndex, persistedSegmentDir, new IndexSpec(), null); - queryableIndex = TestHelper.getTestIndexIO(outputMediumFactory).loadIndex(persistedSegmentDir); + queryableIndex = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex(persistedSegmentDir); } @Test 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 e56c7fa07bae..cced3f214ebc 100644 --- a/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java +++ b/processing/src/test/java/io/druid/query/aggregation/AggregationTestHelper.java @@ -47,7 +47,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.Query; import io.druid.query.QueryPlus; @@ -144,7 +144,7 @@ public static final AggregationTestHelper createGroupByQueryAggregationTestHelpe IndexIO indexIO = new IndexIO( mapper, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -157,7 +157,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, factory.getToolchest(), factory, @@ -201,7 +201,7 @@ public static final AggregationTestHelper createSelectQueryAggregationTestHelper IndexIO indexIO = new IndexIO( mapper, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -214,7 +214,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, toolchest, factory, @@ -242,7 +242,7 @@ public static final AggregationTestHelper createTimeseriesQueryAggregationTestHe IndexIO indexIO = new IndexIO( mapper, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -255,7 +255,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, toolchest, factory, @@ -294,7 +294,7 @@ public ByteBuffer get() IndexIO indexIO = new IndexIO( mapper, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -307,7 +307,7 @@ public int columnCacheSizeBytes() return new AggregationTestHelper( mapper, - new IndexMergerV9(mapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()), + new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()), indexIO, toolchest, factory, 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 897b5b5fdee8..1daeaff48e4d 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -48,7 +48,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.BySegmentQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; @@ -118,7 +118,7 @@ public class GroupByLimitPushDownInsufficientBufferTest ); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -128,7 +128,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } @@ -213,7 +213,7 @@ public void setup() throws Exception indexA, new File(tmpDir, "A"), new IndexSpec(), - OffHeapMemoryOutputMediumFactory.instance() + OffHeapMemorySegmentWriteOutMediumFactory.instance() ); QueryableIndex qindexA = INDEX_IO.loadIndex(fileA); @@ -255,7 +255,7 @@ public void setup() throws Exception indexB, new File(tmpDir, "B"), new IndexSpec(), - OffHeapMemoryOutputMediumFactory.instance() + 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 f74a49d8a7bf..d11f8e62d3f0 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByLimitPushDownMultiNodeMergeTest.java @@ -49,7 +49,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.BySegmentQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; @@ -125,7 +125,7 @@ public class GroupByLimitPushDownMultiNodeMergeTest ); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -135,7 +135,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } private IncrementalIndex makeIncIndex(boolean withRollup) 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 558ecc7c26cf..4df05bc9c681 100644 --- a/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/io/druid/query/groupby/GroupByMultiSegmentTest.java @@ -46,7 +46,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.BySegmentQueryRunner; import io.druid.query.DruidProcessingConfig; import io.druid.query.FinalizeResultsQueryRunner; @@ -114,7 +114,7 @@ public class GroupByMultiSegmentTest ); INDEX_IO = new IndexIO( JSON_MAPPER, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -124,7 +124,7 @@ public int columnCacheSizeBytes() } } ); - INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemoryOutputMediumFactory.instance()); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); } diff --git a/processing/src/test/java/io/druid/segment/AppendTest.java b/processing/src/test/java/io/druid/segment/AppendTest.java index 12a7ef769287..c08048d73d0c 100644 --- a/processing/src/test/java/io/druid/segment/AppendTest.java +++ b/processing/src/test/java/io/druid/segment/AppendTest.java @@ -27,7 +27,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; @@ -113,7 +113,7 @@ public class AppendTest @Before public void setUp() throws Exception { - SchemalessIndexTest schemalessIndexTest = new SchemalessIndexTest(OffHeapMemoryOutputMediumFactory.instance()); + SchemalessIndexTest schemalessIndexTest = new SchemalessIndexTest(OffHeapMemorySegmentWriteOutMediumFactory.instance()); // (1, 2) cover overlapping segments of the form // |------| // |--------| diff --git a/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java b/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java index c9e15a5f834f..785f03c52728 100644 --- a/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java +++ b/processing/src/test/java/io/druid/segment/ConciseBitmapIndexMergerV9Test.java @@ -19,7 +19,7 @@ package io.druid.segment; -import io.druid.output.OutputMediumFactory; +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; @@ -33,7 +33,7 @@ public ConciseBitmapIndexMergerV9Test( CompressionStrategy compressionStrategy, CompressionStrategy dimCompressionStrategy, LongEncodingStrategy longEncodingStrategy, - OutputMediumFactory outputMediumFactory + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { super( @@ -41,8 +41,8 @@ public ConciseBitmapIndexMergerV9Test( compressionStrategy, dimCompressionStrategy, longEncodingStrategy, - outputMediumFactory + segmentWriteOutMediumFactory ); - indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + 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 28fed060ee7f..f7091e2bce4a 100644 --- a/processing/src/test/java/io/druid/segment/EmptyIndexTest.java +++ b/processing/src/test/java/io/druid/segment/EmptyIndexTest.java @@ -24,9 +24,9 @@ import com.google.common.collect.Lists; import io.druid.collections.bitmap.ConciseBitmapFactory; import io.druid.java.util.common.Intervals; -import io.druid.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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; @@ -49,16 +49,16 @@ public class EmptyIndexTest public static Collection constructorFeeder() throws IOException { return ImmutableList.of( - new Object[] {TmpFileOutputMediumFactory.instance()}, - new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} ); } - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; - public EmptyIndexTest(OutputMediumFactory outputMediumFactory) + public EmptyIndexTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Test @@ -83,7 +83,7 @@ public void testEmptyIndex() throws Exception emptyIndex, new ConciseBitmapFactory() ); - TestHelper.getTestIndexMergerV9(outputMediumFactory).merge( + TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory).merge( Lists.newArrayList(emptyIndexAdapter), true, new AggregatorFactory[0], @@ -91,7 +91,7 @@ public void testEmptyIndex() throws Exception new IndexSpec() ); - QueryableIndex emptyQueryableIndex = TestHelper.getTestIndexIO(outputMediumFactory).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 820c89f2147b..5b507be14562 100644 --- a/processing/src/test/java/io/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/io/druid/segment/IndexBuilder.java @@ -26,8 +26,8 @@ import com.google.common.collect.Lists; import io.druid.data.input.InputRow; import io.druid.java.util.common.StringUtils; -import io.druid.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; +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; @@ -52,8 +52,8 @@ public class IndexBuilder private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder() .withMetrics(new CountAggregatorFactory("count")) .build(); - private OutputMediumFactory outputMediumFactory = OffHeapMemoryOutputMediumFactory.instance(); - private IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + 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; @@ -76,10 +76,10 @@ public IndexBuilder schema(IncrementalIndexSchema schema) return this; } - public IndexBuilder outputMediumFactory(OutputMediumFactory outputMediumFactory) + public IndexBuilder segmentWriteOutMediumFactory(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - this.outputMediumFactory = outputMediumFactory; - this.indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; + this.indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); return this; } @@ -123,7 +123,7 @@ public QueryableIndex buildMMappedIndex() Preconditions.checkNotNull(indexMerger, "indexMerger"); Preconditions.checkNotNull(tmpDir, "tmpDir"); try (final IncrementalIndex incrementalIndex = buildIncrementalIndex()) { - return TestHelper.getTestIndexIO(outputMediumFactory).loadIndex( + return TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex( indexMerger.persist( incrementalIndex, new File(tmpDir, StringUtils.format("testIndex-%s", new Random().nextInt(Integer.MAX_VALUE))), @@ -139,14 +139,14 @@ public QueryableIndex buildMMappedIndex() public QueryableIndex buildMMappedMergedIndex() { - IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + 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(outputMediumFactory).loadIndex( + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory).loadIndex( indexMerger.persist( buildIncrementalIndexWithRows( schema, @@ -160,7 +160,7 @@ public QueryableIndex buildMMappedMergedIndex() ) ); } - final QueryableIndex merged = TestHelper.getTestIndexIO(outputMediumFactory).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 132b5211e624..4fefaea23555 100644 --- a/processing/src/test/java/io/druid/segment/IndexIOTest.java +++ b/processing/src/test/java/io/druid/segment/IndexIOTest.java @@ -31,7 +31,7 @@ import io.druid.data.input.impl.DimensionsSpec; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.UOE; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.Aggregator; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.segment.data.CompressionFactory; @@ -329,7 +329,7 @@ public void testRowValidatorEquals() throws Exception { Exception ex = null; try { - TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()).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/IndexMergerTestBase.java b/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java index dd6320727e0c..2434405651b5 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerTestBase.java @@ -41,7 +41,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -88,7 +88,7 @@ public class IndexMergerTestBase protected IndexMerger indexMerger; - @Parameterized.Parameters(name = "{index}: metric compression={0}, dimension compression={1}, long encoding={2}, output medium={3}") + @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( @@ -97,7 +97,7 @@ public static Collection data() EnumSet.allOf(CompressionStrategy.class), ImmutableSet.copyOf(CompressionStrategy.noNoneValues()), EnumSet.allOf(CompressionFactory.LongEncodingStrategy.class), - OutputMediumFactory.builtInFactories() + SegmentWriteOutMediumFactory.builtInFactories() ) ), new Function, Object[]>() { @@ -141,7 +141,7 @@ protected IndexMergerTestBase( CompressionStrategy compressionStrategy, CompressionStrategy dimCompressionStrategy, CompressionFactory.LongEncodingStrategy longEncodingStrategy, - OutputMediumFactory outputMediumFactory + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { this.indexSpec = makeIndexSpec( @@ -150,7 +150,7 @@ protected IndexMergerTestBase( dimCompressionStrategy, longEncodingStrategy ); - this.indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + this.indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); } @Test diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java index cbdc0fc4a35d..aef1e55e2681 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9CompatibilityTest.java @@ -27,9 +27,9 @@ import io.druid.data.input.MapBasedInputRow; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.JodaUtils; -import io.druid.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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.CompressionFactory; @@ -64,8 +64,8 @@ public class IndexMergerV9CompatibilityTest public static Collection constructorFeeder() throws IOException { return ImmutableList.of( - new Object[] {TmpFileOutputMediumFactory.instance()}, - new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} ); } @@ -90,10 +90,10 @@ public static Collection constructorFeeder() throws IOException private final IndexMerger indexMerger; private final IndexIO indexIO; - public IndexMergerV9CompatibilityTest(OutputMediumFactory outputMediumFactory) + public IndexMergerV9CompatibilityTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); - indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); events = new ArrayList<>(); final Map map1 = ImmutableMap.of( diff --git a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java index 8a36102546c0..7dca06cb57f6 100644 --- a/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/io/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -31,7 +31,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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -83,9 +83,9 @@ public class IndexMergerV9WithSpatialIndexTest public static Collection constructorFeeder() throws IOException { List argumentArrays = new ArrayList<>(); - for (OutputMediumFactory outputMediumFactory : OutputMediumFactory.builtInFactories()) { - IndexMergerV9 indexMergerV9 = TestHelper.getTestIndexMergerV9(outputMediumFactory); - IndexIO indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + for (SegmentWriteOutMediumFactory segmentWriteOutMediumFactory : SegmentWriteOutMediumFactory.builtInFactories()) { + IndexMergerV9 indexMergerV9 = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + IndexIO indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); final IndexSpec indexSpec = new IndexSpec(); final IncrementalIndex rtIndex = makeIncrementalIndex(); diff --git a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java index 5891dc619d3d..3fbf954cba84 100644 --- a/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java +++ b/processing/src/test/java/io/druid/segment/QueryableIndexIndexableAdapterTest.java @@ -20,9 +20,9 @@ package io.druid.segment; import com.google.common.collect.ImmutableList; -import io.druid.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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.CompressionFactory; import io.druid.segment.data.CompressionStrategy; @@ -55,8 +55,8 @@ public class QueryableIndexIndexableAdapterTest public static Collection constructorFeeder() throws IOException { return ImmutableList.of( - new Object[] {TmpFileOutputMediumFactory.instance()}, - new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} ); } @@ -68,10 +68,10 @@ public static Collection constructorFeeder() throws IOException private final IndexMerger indexMerger; private final IndexIO indexIO; - public QueryableIndexIndexableAdapterTest(OutputMediumFactory outputMediumFactory) + public QueryableIndexIndexableAdapterTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); - indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); } @Test diff --git a/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java b/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java index d149eede7272..403c5f54ebdf 100644 --- a/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java +++ b/processing/src/test/java/io/druid/segment/RoaringBitmapIndexMergerV9Test.java @@ -19,7 +19,7 @@ package io.druid.segment; -import io.druid.output.OutputMediumFactory; +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; @@ -33,7 +33,7 @@ public RoaringBitmapIndexMergerV9Test( CompressionStrategy compressionStrategy, CompressionStrategy dimCompressionStrategy, LongEncodingStrategy longEncodingStrategy, - OutputMediumFactory outputMediumFactory + SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { super( @@ -41,8 +41,8 @@ public RoaringBitmapIndexMergerV9Test( compressionStrategy, dimCompressionStrategy, longEncodingStrategy, - outputMediumFactory + segmentWriteOutMediumFactory ); - indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + 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 5bafa9493d39..f31c4a75f3eb 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessIndexTest.java @@ -35,7 +35,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -102,10 +102,10 @@ public class SchemalessIndexTest private final IndexMerger indexMerger; private final IndexIO indexIO; - public SchemalessIndexTest(OutputMediumFactory outputMediumFactory) + public SchemalessIndexTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); - indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); } public static IncrementalIndex getIncrementalIndex() diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java index 34376af315d6..a08c49b308f8 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestFullTest.java @@ -30,9 +30,9 @@ 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.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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; @@ -82,8 +82,8 @@ public class SchemalessTestFullTest public static Collection constructorFeeder() throws IOException { return ImmutableList.of( - new Object[] {TmpFileOutputMediumFactory.instance()}, - new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} ); } @@ -116,9 +116,9 @@ public static Collection constructorFeeder() throws IOException Arrays.asList(Intervals.of("1970-01-01T00:00:00.000Z/2020-01-01T00:00:00.000Z")) ); - public SchemalessTestFullTest(OutputMediumFactory outputMediumFactory) + public SchemalessTestFullTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - schemalessIndexTest = new SchemalessIndexTest(outputMediumFactory); + schemalessIndexTest = new SchemalessIndexTest(segmentWriteOutMediumFactory); } @Test diff --git a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java index dc6fa9861f26..d12a8225e6ff 100644 --- a/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java +++ b/processing/src/test/java/io/druid/segment/SchemalessTestSimpleTest.java @@ -26,7 +26,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; @@ -76,8 +76,8 @@ public class SchemalessTestSimpleTest public static Collection constructorFeeder() throws IOException { List argumentArrays = new ArrayList<>(); - for (OutputMediumFactory outputMediumFactory : OutputMediumFactory.builtInFactories()) { - SchemalessIndexTest schemalessIndexTest = new SchemalessIndexTest(outputMediumFactory); + 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(); diff --git a/processing/src/test/java/io/druid/segment/TestHelper.java b/processing/src/test/java/io/druid/segment/TestHelper.java index 55745ac348c5..7da66da3b590 100644 --- a/processing/src/test/java/io/druid/segment/TestHelper.java +++ b/processing/src/test/java/io/druid/segment/TestHelper.java @@ -29,7 +29,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Result; import io.druid.query.expression.TestExprMacroTable; import io.druid.query.timeseries.TimeseriesResultValue; @@ -49,16 +49,16 @@ public class TestHelper { private static final ObjectMapper JSON_MAPPER = getJsonMapper(); - public static IndexMergerV9 getTestIndexMergerV9(OutputMediumFactory outputMediumFactory) + public static IndexMergerV9 getTestIndexMergerV9(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - return new IndexMergerV9(JSON_MAPPER, getTestIndexIO(outputMediumFactory), outputMediumFactory); + return new IndexMergerV9(JSON_MAPPER, getTestIndexIO(segmentWriteOutMediumFactory), segmentWriteOutMediumFactory); } - public static IndexIO getTestIndexIO(OutputMediumFactory outputMediumFactory) + public static IndexIO getTestIndexIO(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { return new IndexIO( JSON_MAPPER, - outputMediumFactory, + segmentWriteOutMediumFactory, new ColumnConfig() { @Override diff --git a/processing/src/test/java/io/druid/segment/TestIndex.java b/processing/src/test/java/io/druid/segment/TestIndex.java index edd181f90efb..a6b03a6dd0d6 100644 --- a/processing/src/test/java/io/druid/segment/TestIndex.java +++ b/processing/src/test/java/io/druid/segment/TestIndex.java @@ -37,7 +37,7 @@ 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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.DoubleMaxAggregatorFactory; import io.druid.query.aggregation.DoubleMinAggregatorFactory; @@ -137,8 +137,8 @@ public class TestIndex private static final IndexSpec indexSpec = new IndexSpec(); private static final IndexMerger INDEX_MERGER = - TestHelper.getTestIndexMergerV9(OffHeapMemoryOutputMediumFactory.instance()); - private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()); + TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()); + private static final IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()); static { if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { 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 c5833e34dc49..5eaa9e55a29d 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedFloatsSerdeTest.java @@ -23,7 +23,7 @@ import com.google.common.primitives.Floats; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.output.OffHeapMemoryOutputMedium; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.Assert; import org.junit.Test; @@ -108,7 +108,7 @@ public void testChunkSerde() throws Exception public void testWithValues(float[] values) throws Exception { FloatSupplierSerializer serializer = CompressionFactory.getFloatSerializer( - new OffHeapMemoryOutputMedium(), + new OffHeapMemorySegmentWriteOutMedium(), "test", order, compressionStrategy 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 a0fd7e7fb701..3a2441e4f51e 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedIntsIndexedWriterTest.java @@ -29,9 +29,9 @@ 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.output.OffHeapMemoryOutputMedium; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +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; @@ -57,7 +57,7 @@ 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 OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); + private final SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); private final CompressionStrategy compressionStrategy; private final ByteOrder byteOrder; private final Random rand = new Random(0); @@ -101,7 +101,7 @@ public void setUp() throws Exception @After public void tearDown() throws Exception { - outputMedium.close(); + segmentWriteOutMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -117,29 +117,29 @@ private void checkSerializedSizeAndData(int chunkFactor) throws Exception FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( - outputMedium, "test", chunkFactor, byteOrder, compressionStrategy + segmentWriteOutMedium, "test", chunkFactor, byteOrder, compressionStrategy ); CompressedIntsIndexedSupplier supplierFromList = CompressedIntsIndexedSupplier.fromList( IntArrayList.wrap(vals), chunkFactor, byteOrder, compressionStrategy, - outputMedium.getCloser() + segmentWriteOutMedium.getCloser() ); writer.open(); for (int val : vals) { writer.add(val); } long writtenLength = writer.getSerializedSize(); - final OutputBytes outputBytes = outputMedium.makeOutputBytes(); - writer.writeTo(outputBytes, smoosher); + 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(outputBytes.asInputStream())), + ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())), byteOrder ); IndexedInts indexedInts = supplierFromByteBuffer.get(); @@ -192,12 +192,12 @@ private void checkV2SerializedSizeAndData(int chunkFactor) throws Exception FileSmoosher smoosher = new FileSmoosher(tmpDirectory); CompressedIntsIndexedWriter writer = new CompressedIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, chunkFactor, byteOrder, compressionStrategy, GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, "test", compressionStrategy, Longs.BYTES * 10000 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 f0c970fa2a65..a628de90f5f6 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java @@ -23,7 +23,7 @@ import com.google.common.primitives.Longs; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.CloseQuietly; -import io.druid.output.OffHeapMemoryOutputMedium; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import it.unimi.dsi.fastutil.ints.IntArrays; import org.junit.Assert; import org.junit.Test; @@ -130,7 +130,7 @@ public void testWithValues(long[] values) throws Exception public void testValues(long[] values) throws Exception { LongSupplierSerializer serializer = CompressionFactory.getLongSerializer( - new OffHeapMemoryOutputMedium(), + new OffHeapMemorySegmentWriteOutMedium(), "test", order, encodingStrategy, 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 1a3490c8a26a..23b605c6e471 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -31,9 +31,9 @@ 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.output.OffHeapMemoryOutputMedium; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; +import io.druid.segment.writeout.WriteOutBytes; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.CompressedVSizeIndexedV3Supplier; import org.apache.commons.io.FileUtils; import org.apache.commons.io.IOUtils; @@ -116,13 +116,13 @@ private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFac { FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); - try (OutputMedium outputMedium = new OffHeapMemoryOutputMedium()) { + try (SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium()) { int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( - outputMedium, "offset", offsetChunkFactor, byteOrder, compressionStrategy + segmentWriteOutMedium, "offset", offsetChunkFactor, byteOrder, compressionStrategy ); CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter( - outputMedium, "value", maxValue, valueChunkFactor, byteOrder, compressionStrategy + segmentWriteOutMedium, "value", maxValue, valueChunkFactor, byteOrder, compressionStrategy ); CompressedVSizeIndexedV3Writer writer = new CompressedVSizeIndexedV3Writer(offsetWriter, valueWriter); CompressedVSizeIndexedV3Supplier supplierFromIterable = CompressedVSizeIndexedV3Supplier.fromIterable( @@ -131,22 +131,22 @@ private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFac maxValue, byteOrder, compressionStrategy, - outputMedium.getCloser() + segmentWriteOutMedium.getCloser() ); writer.open(); for (int[] val : vals) { writer.add(val); } long writtenLength = writer.getSerializedSize(); - final OutputBytes outputBytes = outputMedium.makeOutputBytes(); - writer.writeTo(outputBytes, smoosher); + 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(outputBytes.asInputStream())), + ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())), byteOrder, null ); @@ -230,14 +230,14 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF FileSmoosher smoosher = new FileSmoosher(tmpDirectory); int maxValue = vals.size() > 0 ? getMaxValue(vals) : 0; - try (OutputMedium outputMedium = new OffHeapMemoryOutputMedium()) { + try (SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium()) { CompressedIntsIndexedWriter offsetWriter = new CompressedIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, offsetChunkFactor, byteOrder, compressionStrategy, GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, "offset", compressionStrategy, Longs.BYTES * 250000 @@ -245,13 +245,13 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF ); GenericIndexedWriter genericIndexed = GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, "value", compressionStrategy, Longs.BYTES * 250000 ); CompressedVSizeIntsIndexedWriter valueWriter = new CompressedVSizeIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, maxValue, valueChunkFactor, byteOrder, 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 bb7782ef950f..bc2798565979 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIntsIndexedWriterTest.java @@ -29,9 +29,9 @@ 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.output.OffHeapMemoryOutputMedium; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +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; @@ -55,7 +55,7 @@ public class CompressedVSizeIntsIndexedWriterTest { private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; - private final OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); + private final SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); private final CompressionStrategy compressionStrategy; private final ByteOrder byteOrder; private final Random rand = new Random(0); @@ -98,7 +98,7 @@ public void setUp() throws Exception @After public void tearDown() throws Exception { - outputMedium.close(); + segmentWriteOutMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -114,7 +114,7 @@ private void checkSerializedSizeAndData(int chunkSize) throws Exception FileSmoosher smoosher = new FileSmoosher(FileUtils.getTempDirectory()); CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, "test", vals.length > 0 ? Ints.max(vals) : 0, chunkSize, @@ -127,22 +127,22 @@ private void checkSerializedSizeAndData(int chunkSize) throws Exception chunkSize, byteOrder, compressionStrategy, - outputMedium.getCloser() + segmentWriteOutMedium.getCloser() ); writer.open(); for (int val : vals) { writer.add(val); } long writtenLength = writer.getSerializedSize(); - final OutputBytes outputBytes = outputMedium.makeOutputBytes(); - writer.writeTo(outputBytes, smoosher); + 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(outputBytes.asInputStream())), + ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())), byteOrder ); IndexedInts indexedInts = supplierFromByteBuffer.get(); @@ -187,13 +187,13 @@ private void checkV2SerializedSizeAndData(int chunkSize) throws Exception FileSmoosher smoosher = new FileSmoosher(tmpDirectory); GenericIndexedWriter genericIndexed = GenericIndexedWriter.ofCompressedByteBuffers( - outputMedium, + segmentWriteOutMedium, "test", compressionStrategy, Longs.BYTES * 10000 ); CompressedVSizeIntsIndexedWriter writer = new CompressedVSizeIntsIndexedWriter( - outputMedium, + segmentWriteOutMedium, vals.length > 0 ? Ints.max(vals) : 0, chunkSize, byteOrder, diff --git a/processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java b/processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java index 6f6888852244..ec7f5d8e69b8 100644 --- a/processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/GenericIndexedStringWriterTest.java @@ -19,7 +19,7 @@ package io.druid.segment.data; -import io.druid.output.OnHeapMemoryOutputMedium; +import io.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; import org.junit.Assert; import org.junit.Test; @@ -34,9 +34,9 @@ public class GenericIndexedStringWriterTest @Test public void testRandomAccess() throws IOException { - OnHeapMemoryOutputMedium outputMedium = new OnHeapMemoryOutputMedium(); + OnHeapMemorySegmentWriteOutMedium segmentWriteOutMedium = new OnHeapMemorySegmentWriteOutMedium(); GenericIndexedWriter writer = new GenericIndexedWriter<>( - outputMedium, + segmentWriteOutMedium, "test", GenericIndexed.STRING_STRATEGY ); 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 d110225570ba..2649b4ca8eaf 100644 --- a/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/VSizeIndexedIntsWriterTest.java @@ -20,9 +20,9 @@ package io.druid.segment.data; import com.google.common.primitives.Ints; -import io.druid.output.OffHeapMemoryOutputMedium; -import io.druid.output.OutputBytes; -import io.druid.output.OutputMedium; +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; @@ -39,7 +39,7 @@ public class VSizeIndexedIntsWriterTest { private static final int[] MAX_VALUES = new int[]{0xFF, 0xFFFF, 0xFFFFFF, 0x0FFFFFFF}; - private final OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); + private final SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); private final Random rand = new Random(0); private int[] vals; @@ -52,7 +52,7 @@ public void setUp() throws Exception @After public void tearDown() throws Exception { - outputMedium.close(); + segmentWriteOutMedium.close(); } private void generateVals(final int totalSize, final int maxValue) throws IOException @@ -66,7 +66,7 @@ private void generateVals(final int totalSize, final int maxValue) throws IOExce private void checkSerializedSizeAndData() throws Exception { int maxValue = vals.length == 0 ? 0 : Ints.max(vals); - VSizeIndexedIntsWriter writer = new VSizeIndexedIntsWriter(outputMedium, maxValue); + VSizeIndexedIntsWriter writer = new VSizeIndexedIntsWriter(segmentWriteOutMedium, maxValue); VSizeIndexedInts intsFromList = VSizeIndexedInts.fromList( IntArrayList.wrap(vals), maxValue @@ -76,14 +76,14 @@ private void checkSerializedSizeAndData() throws Exception writer.add(val); } long writtenLength = writer.getSerializedSize(); - OutputBytes outputBytes = outputMedium.makeOutputBytes(); - writer.writeTo(outputBytes, null); + 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(outputBytes.asInputStream())) + 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/filter/BaseFilterTest.java b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java index 8e309d5b3237..96ca5fc4c5be 100644 --- a/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/BaseFilterTest.java @@ -34,9 +34,9 @@ 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.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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; @@ -184,9 +184,9 @@ public static Collection makeConstructors() "roaring", new RoaringBitmapSerdeFactory(true) ); - final Map outputMediumFactories = ImmutableMap.of( - "tmpFile output medium", TmpFileOutputMediumFactory.instance(), - "off-heap memory output medium", OffHeapMemoryOutputMediumFactory.instance() + 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( @@ -250,7 +250,8 @@ public void close() throws IOException ); for (Map.Entry bitmapSerdeFactoryEntry : bitmapSerdeFactories.entrySet()) { - for (Map.Entry outputMediumFactoryEntry : outputMediumFactories.entrySet()) { + for (Map.Entry segmentWriteOutMediumFactoryEntry : + segmentWriteOutMediumFactories.entrySet()) { for (Map.Entry>> finisherEntry : finishers.entrySet()) { for (boolean cnf : ImmutableList.of(false, true)) { @@ -258,18 +259,14 @@ public void close() throws IOException final String testName = StringUtils.format( "bitmaps[%s], indexMerger[%s], finisher[%s], optimize[%s]", bitmapSerdeFactoryEntry.getKey(), - outputMediumFactoryEntry.getKey(), + segmentWriteOutMediumFactoryEntry.getKey(), finisherEntry.getKey(), optimize ); - final IndexBuilder indexBuilder = IndexBuilder.create() - .indexSpec(new IndexSpec( - bitmapSerdeFactoryEntry.getValue(), - null, - null, - null - )) - .outputMediumFactory(outputMediumFactoryEntry.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 be0303242ec7..3ff7337942cf 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterBonusTest.java @@ -32,7 +32,7 @@ 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.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -98,9 +98,9 @@ public SpatialFilterBonusTest(Segment segment) public static Collection constructorFeeder() throws IOException { List argumentArrays = new ArrayList<>(); - for (OutputMediumFactory outputMediumFactory : OutputMediumFactory.builtInFactories()) { - IndexMerger indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); - IndexIO indexIO = TestHelper.getTestIndexIO(outputMediumFactory); + 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); 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 00545a377042..31311cb15f2e 100644 --- a/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java +++ b/processing/src/test/java/io/druid/segment/filter/SpatialFilterTest.java @@ -31,7 +31,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.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.QueryPlus; @@ -74,8 +74,8 @@ @RunWith(Parameterized.class) public class SpatialFilterTest { - private static IndexMerger INDEX_MERGER = TestHelper.getTestIndexMergerV9(OffHeapMemoryOutputMediumFactory.instance()); - private static IndexIO INDEX_IO = TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()); + 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"); 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 51901f3641dc..5df84c4ddd87 100644 --- a/processing/src/test/java/io/druid/segment/loading/SegmentizerFactoryTest.java +++ b/processing/src/test/java/io/druid/segment/loading/SegmentizerFactoryTest.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.SegmentizerModule; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.segment.IndexIO; import io.druid.segment.column.ColumnConfig; import org.junit.Assert; @@ -45,7 +45,7 @@ public void testFactory() throws IOException mapper.registerModule(new SegmentizerModule()); IndexIO indexIO = new IndexIO( mapper, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override 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 edf1f402907e..602868d48499 100644 --- a/processing/src/test/java/io/druid/segment/serde/HyperUniquesSerdeForTest.java +++ b/processing/src/test/java/io/druid/segment/serde/HyperUniquesSerdeForTest.java @@ -24,7 +24,7 @@ import com.metamx.common.StringUtils; import io.druid.data.input.InputRow; import io.druid.hll.HyperLogLogCollector; -import io.druid.output.OutputMedium; +import io.druid.segment.writeout.SegmentWriteOutMedium; import io.druid.segment.GenericColumnSerializer; import io.druid.segment.column.ColumnBuilder; import io.druid.segment.data.GenericIndexed; @@ -152,10 +152,10 @@ public int compare(HyperLogLogCollector o1, HyperLogLogCollector o2) } @Override - public GenericColumnSerializer getSerializer(OutputMedium outputMedium, String metric) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String metric) { return LargeColumnSupportedComplexColumnSerializer.createWithColumnSize( - outputMedium, + 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 d406968e1a67..de710e3c8a17 100644 --- a/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java +++ b/processing/src/test/java/io/druid/segment/serde/LargeColumnSupportedComplexColumnSerializerTest.java @@ -27,8 +27,8 @@ 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.output.OffHeapMemoryOutputMedium; -import io.druid.output.OutputMedium; +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; @@ -60,11 +60,11 @@ public void testSanity() throws IOException for (int aCase : cases) { File tmpFile = FileUtils.getTempDirectory(); HyperLogLogCollector baseCollector = HyperLogLogCollector.makeLatestCollector(); - try (OutputMedium outputMedium = new OffHeapMemoryOutputMedium(); + try (SegmentWriteOutMedium segmentWriteOutMedium = new OffHeapMemorySegmentWriteOutMedium(); FileSmoosher v9Smoosher = new FileSmoosher(tmpFile)) { LargeColumnSupportedComplexColumnSerializer serializer = LargeColumnSupportedComplexColumnSerializer - .createWithColumnSize(outputMedium, "test", serde.getObjectStrategy(), columnSize); + .createWithColumnSize(segmentWriteOutMedium, "test", serde.getObjectStrategy(), columnSize); serializer.open(); for (int i = 0; i < aCase; i++) { diff --git a/processing/src/test/java/io/druid/output/OutputBytesTest.java b/processing/src/test/java/io/druid/segment/writeout/WriteOutBytesTest.java similarity index 50% rename from processing/src/test/java/io/druid/output/OutputBytesTest.java rename to processing/src/test/java/io/druid/segment/writeout/WriteOutBytesTest.java index a2bd49e39ec5..6b62ad30c862 100644 --- a/processing/src/test/java/io/druid/output/OutputBytesTest.java +++ b/processing/src/test/java/io/druid/segment/writeout/WriteOutBytesTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.output; +package io.druid.segment.writeout; import com.google.common.io.Files; import com.google.common.primitives.Ints; @@ -36,54 +36,54 @@ import java.util.Collection; @RunWith(Parameterized.class) -public class OutputBytesTest +public class WriteOutBytesTest { @Parameterized.Parameters public static Collection constructorFeeder() throws IOException { return Arrays.asList( - new Object[] {new TmpFileOutputMedium(Files.createTempDir())}, - new Object[] {new OffHeapMemoryOutputMedium()}, - new Object[] {new OnHeapMemoryOutputMedium()} + new Object[] {new TmpFileSegmentWriteOutMedium(Files.createTempDir())}, + new Object[] {new OffHeapMemorySegmentWriteOutMedium()}, + new Object[] {new OnHeapMemorySegmentWriteOutMedium()} ); } - private final OutputMedium outputMedium; + private final SegmentWriteOutMedium segmentWriteOutMedium; - public OutputBytesTest(OutputMedium outputMedium) + public WriteOutBytesTest(SegmentWriteOutMedium segmentWriteOutMedium) { - this.outputMedium = outputMedium; + this.segmentWriteOutMedium = segmentWriteOutMedium; } @Test - public void testOutputBytes() throws IOException + public void testWriteOutBytes() throws IOException { - OutputBytes outputBytes = outputMedium.makeOutputBytes(); + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); - outputBytes.write('1'); - verifyContents(outputBytes, "1"); + writeOutBytes.write('1'); + verifyContents(writeOutBytes, "1"); - outputBytes.writeInt(Ints.fromBytes((byte) '2', (byte) '3', (byte) '4', (byte) '5')); - verifyContents(outputBytes, "12345"); + writeOutBytes.writeInt(Ints.fromBytes((byte) '2', (byte) '3', (byte) '4', (byte) '5')); + verifyContents(writeOutBytes, "12345"); - outputBytes.write(new byte[] {'a'}); - verifyContents(outputBytes, "12345a"); + writeOutBytes.write(new byte[] {'a'}); + verifyContents(writeOutBytes, "12345a"); - outputBytes.write(new byte[] {'a', 'b', 'c'}, 1, 1); - verifyContents(outputBytes, "12345ab"); + writeOutBytes.write(new byte[] {'a', 'b', 'c'}, 1, 1); + verifyContents(writeOutBytes, "12345ab"); ByteBuffer bb = ByteBuffer.wrap(new byte[]{'a', 'b', 'c'}); bb.position(2); - outputBytes.write(bb); + writeOutBytes.write(bb); Assert.assertEquals(3, bb.position()); - verifyContents(outputBytes, "12345abc"); + verifyContents(writeOutBytes, "12345abc"); } - private void verifyContents(OutputBytes outputBytes, String expected) throws IOException + private void verifyContents(WriteOutBytes writeOutBytes, String expected) throws IOException { - Assert.assertEquals(expected, IOUtils.toString(outputBytes.asInputStream(), StandardCharsets.US_ASCII)); - ByteBuffer bb = ByteBuffer.allocate((int) outputBytes.size()); - outputBytes.readFully(0, bb); + 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)); } @@ -91,15 +91,15 @@ private void verifyContents(OutputBytes outputBytes, String expected) throws IOE @Test public void testCrossBufferRandomAccess() throws IOException { - OutputBytes outputBytes = outputMedium.makeOutputBytes(); - for (int i = 0; i < ByteBufferOutputBytes.BUFFER_SIZE; i++) { - outputBytes.write('0'); + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + for (int i = 0; i < ByteBufferWriteOutBytes.BUFFER_SIZE; i++) { + writeOutBytes.write('0'); } - outputBytes.write('1'); - outputBytes.write('2'); - outputBytes.write('3'); + writeOutBytes.write('1'); + writeOutBytes.write('2'); + writeOutBytes.write('3'); ByteBuffer bb = ByteBuffer.allocate(4); - outputBytes.readFully(ByteBufferOutputBytes.BUFFER_SIZE - 1, bb); + writeOutBytes.readFully(ByteBufferWriteOutBytes.BUFFER_SIZE - 1, bb); bb.flip(); Assert.assertEquals("0123", StringUtils.fromUtf8(bb)); } @@ -107,16 +107,16 @@ public void testCrossBufferRandomAccess() throws IOException @Test(expected = BufferUnderflowException.class) public void testReadFullyUnderflow() throws IOException { - OutputBytes outputBytes = outputMedium.makeOutputBytes(); - outputBytes.write('1'); - outputBytes.readFully(0, ByteBuffer.allocate(2)); + WriteOutBytes writeOutBytes = segmentWriteOutMedium.makeWriteOutBytes(); + writeOutBytes.write('1'); + writeOutBytes.readFully(0, ByteBuffer.allocate(2)); } @Test public void testReadFullyEmptyAtTheEnd() throws IOException { - OutputBytes outputBytes = outputMedium.makeOutputBytes(); - outputBytes.write('1'); - outputBytes.readFully(1, ByteBuffer.allocate(0)); + 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 20b2761d6bad..8f80c1111996 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -63,7 +63,7 @@ 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.output.OutputMediumModule; +import io.druid.segment.writeout.SegmentWriteOutMediumModule; import io.druid.server.emitter.EmitterModule; import io.druid.server.initialization.AuthenticatorHttpClientWrapperModule; import io.druid.server.initialization.AuthenticatorMapperModule; @@ -361,7 +361,7 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new AnnouncerModule(), new AWSModule(), new MetricsModule(), - new OutputMediumModule(), + 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 e044cfae16bf..32c2f5ec962e 100644 --- a/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java +++ b/server/src/main/java/io/druid/segment/indexing/RealtimeTuningConfig.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.io.Files; -import io.druid.output.OutputMediumFactory; +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; @@ -96,7 +96,7 @@ public static RealtimeTuningConfig makeDefaultTuningConfig(final @Nullable File private final long handoffConditionTimeout; private final long alertTimeout; @Nullable - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; @JsonCreator public RealtimeTuningConfig( @@ -116,7 +116,7 @@ public RealtimeTuningConfig( @JsonProperty("reportParseExceptions") Boolean reportParseExceptions, @JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout, @JsonProperty("alertTimeout") Long alertTimeout, - @JsonProperty("outputMediumFactory") @Nullable OutputMediumFactory outputMediumFactory + @JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory ) { this.maxRowsInMemory = maxRowsInMemory == null ? defaultMaxRowsInMemory : maxRowsInMemory; @@ -144,7 +144,7 @@ public RealtimeTuningConfig( this.alertTimeout = alertTimeout == null ? defaultAlertTimeout : alertTimeout; Preconditions.checkArgument(this.alertTimeout >= 0, "alertTimeout must be >= 0"); - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Override @@ -250,9 +250,9 @@ public long getAlertTimeout() @Override @JsonProperty @Nullable - public OutputMediumFactory getOutputMediumFactory() + public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory() { - return outputMediumFactory; + return segmentWriteOutMediumFactory; } public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) @@ -273,7 +273,7 @@ public RealtimeTuningConfig withVersioningPolicy(VersioningPolicy policy) reportParseExceptions, handoffConditionTimeout, alertTimeout, - outputMediumFactory + segmentWriteOutMediumFactory ); } @@ -295,7 +295,7 @@ public RealtimeTuningConfig withBasePersistDirectory(File dir) reportParseExceptions, handoffConditionTimeout, alertTimeout, - outputMediumFactory + 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 631e27344d99..286470cae761 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,7 +19,7 @@ package io.druid.segment.realtime.appenderator; -import io.druid.output.OutputMediumFactory; +import io.druid.segment.writeout.SegmentWriteOutMediumFactory; import io.druid.segment.IndexSpec; import org.joda.time.Period; @@ -41,5 +41,5 @@ public interface AppenderatorConfig File getBasePersistDirectory(); @Nullable - OutputMediumFactory getOutputMediumFactory(); + 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 73257663c8af..9475f0552322 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 @@ -597,7 +597,7 @@ private DataSegment mergeAndPush(final SegmentIdentifier identifier, final Sink schema.getAggregators(), mergedTarget, tuningConfig.getIndexSpec(), - tuningConfig.getOutputMediumFactory() + tuningConfig.getSegmentWriteOutMediumFactory() ); } catch (Throwable t) { @@ -1043,7 +1043,7 @@ private int persistHydrant(FireHydrant indexToPersist, SegmentIdentifier identif identifier.getInterval(), new File(persistDir, String.valueOf(indexToPersist.getCount())), indexSpec, - tuningConfig.getOutputMediumFactory() + tuningConfig.getSegmentWriteOutMediumFactory() ); indexToPersist.swapSegment( 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 25744611325d..c0bdb2eb6339 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 @@ -428,7 +428,7 @@ public void doRun() schema.getAggregators(), mergedTarget, config.getIndexSpec(), - config.getOutputMediumFactory() + config.getSegmentWriteOutMediumFactory() ); } catch (Throwable t) { @@ -943,7 +943,7 @@ protected int persistHydrant( interval, new File(computePersistDir(schema, interval), String.valueOf(indexToPersist.getCount())), indexSpec, - config.getOutputMediumFactory() + config.getSegmentWriteOutMediumFactory() ); indexToPersist.swapSegment( 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 8f6e35742ad9..ffacdb5f6a96 100644 --- a/server/src/test/java/io/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java +++ b/server/src/test/java/io/druid/segment/loading/SegmentLoaderLocalCacheManagerTest.java @@ -28,9 +28,9 @@ import com.metamx.emitter.EmittingLogger; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.Intervals; -import io.druid.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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; @@ -55,8 +55,8 @@ public class SegmentLoaderLocalCacheManagerTest public static Collection constructorFeeder() throws IOException { return ImmutableList.of( - new Object[] {TmpFileOutputMediumFactory.instance()}, - new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} ); } @@ -64,12 +64,12 @@ public static Collection constructorFeeder() throws IOException public final TemporaryFolder tmpFolder = new TemporaryFolder(); private final ObjectMapper jsonMapper; - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private File localSegmentCacheFolder; private SegmentLoaderLocalCacheManager manager; - public SegmentLoaderLocalCacheManagerTest(OutputMediumFactory outputMediumFactory) + public SegmentLoaderLocalCacheManagerTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { jsonMapper = new DefaultObjectMapper(); jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local")); @@ -79,7 +79,7 @@ public SegmentLoaderLocalCacheManagerTest(OutputMediumFactory outputMediumFactor new LocalDataSegmentPuller() ) ); - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Before @@ -95,7 +95,7 @@ public void setUp() throws Exception locations.add(locationConfig); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(outputMediumFactory), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -169,7 +169,7 @@ public void testRetrySuccessAtFirstLocation() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(outputMediumFactory), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -222,7 +222,7 @@ public void testRetrySuccessAtSecondLocation() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(outputMediumFactory), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -277,7 +277,7 @@ public void testRetryAllFail() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(outputMediumFactory), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), new SegmentLoaderConfig().withLocations(locations), jsonMapper ); @@ -331,7 +331,7 @@ public void testEmptyToFullOrder() throws Exception locations.add(locationConfig2); manager = new SegmentLoaderLocalCacheManager( - TestHelper.getTestIndexIO(outputMediumFactory), + 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 91e0fab5d66b..399688ddad31 100644 --- a/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java +++ b/server/src/test/java/io/druid/segment/realtime/FireDepartmentTest.java @@ -29,7 +29,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +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 +110,8 @@ public void testSerde() throws Exception null, null, null, - TestHelper.getTestIndexMergerV9(OffHeapMemoryOutputMediumFactory.instance()), - TestHelper.getTestIndexIO(OffHeapMemoryOutputMediumFactory.instance()), + 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/appenderator/AppenderatorTester.java b/server/src/test/java/io/druid/segment/realtime/appenderator/AppenderatorTester.java index 5e503f6fdd74..c51470689eee 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 @@ -36,7 +36,7 @@ import io.druid.data.input.impl.TimestampSpec; import io.druid.jackson.DefaultObjectMapper; import io.druid.java.util.common.granularity.Granularities; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.IntervalChunkingQueryRunnerDecorator; import io.druid.query.Query; @@ -158,7 +158,7 @@ public AppenderatorTester( indexIO = new IndexIO( objectMapper, - OffHeapMemoryOutputMediumFactory.instance(), + OffHeapMemorySegmentWriteOutMediumFactory.instance(), new ColumnConfig() { @Override @@ -168,7 +168,7 @@ public int columnCacheSizeBytes() } } ); - indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemoryOutputMediumFactory.instance()); + indexMerger = new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); emitter = new ServiceEmitter( "test", 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 5403cbfee3e1..426f2af86f95 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,9 +33,9 @@ import io.druid.hll.HyperLogLogCollector; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; -import io.druid.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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; @@ -99,8 +99,8 @@ public class IngestSegmentFirehoseTest public static Collection constructorFeeder() throws IOException { return ImmutableList.of( - new Object[] {TmpFileOutputMediumFactory.instance()}, - new Object[] {OffHeapMemoryOutputMediumFactory.instance()} + new Object[] {TmpFileSegmentWriteOutMediumFactory.instance()}, + new Object[] {OffHeapMemorySegmentWriteOutMediumFactory.instance()} ); } @@ -110,10 +110,10 @@ public static Collection constructorFeeder() throws IOException private final IndexIO indexIO; private final IndexMerger indexMerger; - public IngestSegmentFirehoseTest(OutputMediumFactory outputMediumFactory) + public IngestSegmentFirehoseTest(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { - indexIO = TestHelper.getTestIndexIO(outputMediumFactory); - indexMerger = TestHelper.getTestIndexMergerV9(outputMediumFactory); + indexIO = TestHelper.getTestIndexIO(segmentWriteOutMediumFactory); + indexMerger = TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory); } @Test 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 bcca54fb2c31..2c3a83b4367f 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,9 +39,9 @@ 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.output.OffHeapMemoryOutputMediumFactory; -import io.druid.output.OutputMediumFactory; -import io.druid.output.TmpFileOutputMediumFactory; +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; @@ -87,7 +87,7 @@ @RunWith(Parameterized.class) public class RealtimePlumberSchoolTest { - @Parameterized.Parameters(name = "rejectionPolicy = {0}, outputMediumFactory = {1}") + @Parameterized.Parameters(name = "rejectionPolicy = {0}, segmentWriteOutMediumFactory = {1}") public static Collection constructorFeeder() throws IOException { final RejectionPolicyFactory[] rejectionPolicies = new RejectionPolicyFactory[]{ @@ -97,14 +97,14 @@ public static Collection constructorFeeder() throws IOException final List constructors = Lists.newArrayList(); for (RejectionPolicyFactory rejectionPolicy : rejectionPolicies) { - constructors.add(new Object[]{rejectionPolicy, OffHeapMemoryOutputMediumFactory.instance()}); - constructors.add(new Object[]{rejectionPolicy, TmpFileOutputMediumFactory.instance()}); + constructors.add(new Object[]{rejectionPolicy, OffHeapMemorySegmentWriteOutMediumFactory.instance()}); + constructors.add(new Object[]{rejectionPolicy, TmpFileSegmentWriteOutMediumFactory.instance()}); } return constructors; } private final RejectionPolicyFactory rejectionPolicy; - private final OutputMediumFactory outputMediumFactory; + private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory; private RealtimePlumber plumber; private RealtimePlumberSchool realtimePlumberSchool; private DataSegmentAnnouncer announcer; @@ -119,10 +119,10 @@ public static Collection constructorFeeder() throws IOException private FireDepartmentMetrics metrics; private File tmpDir; - public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, OutputMediumFactory outputMediumFactory) + public RealtimePlumberSchoolTest(RejectionPolicyFactory rejectionPolicy, SegmentWriteOutMediumFactory segmentWriteOutMediumFactory) { this.rejectionPolicy = rejectionPolicy; - this.outputMediumFactory = outputMediumFactory; + this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory; } @Before @@ -220,8 +220,8 @@ public void setUp() throws Exception segmentPublisher, handoffNotifierFactory, MoreExecutors.sameThreadExecutor(), - TestHelper.getTestIndexMergerV9(outputMediumFactory), - TestHelper.getTestIndexIO(outputMediumFactory), + TestHelper.getTestIndexMergerV9(segmentWriteOutMediumFactory), + TestHelper.getTestIndexIO(segmentWriteOutMediumFactory), MapCache.create(0), FireDepartmentTest.NO_CACHE_CONFIG, TestHelper.getJsonMapper() 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 c6f7b9f6926c..ea7ce8352c04 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,7 +24,7 @@ import com.google.common.collect.ImmutableSet; import io.druid.data.input.InputRow; import io.druid.java.util.common.Intervals; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.DoubleSumAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; @@ -88,7 +88,7 @@ public void setUp() throws Exception final File tmpDir = temporaryFolder.newFolder(); final QueryableIndex index1 = IndexBuilder.create() .tmpDir(new File(tmpDir, "1")) - .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema( new IncrementalIndexSchema.Builder() .withMetrics( @@ -104,7 +104,7 @@ public void setUp() throws Exception final QueryableIndex index2 = IndexBuilder.create() .tmpDir(new File(tmpDir, "2")) - .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) + .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 895c45161858..12293a56ec39 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 @@ -43,7 +43,7 @@ import io.druid.guice.ExpressionModule; import io.druid.guice.annotations.Json; import io.druid.math.expr.ExprMacroTable; -import io.druid.output.OffHeapMemoryOutputMediumFactory; +import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import io.druid.query.DefaultGenericQueryMetricsFactory; import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.DruidProcessingConfig; @@ -344,14 +344,14 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker(final File tmp { final QueryableIndex index1 = IndexBuilder.create() .tmpDir(new File(tmpDir, "1")) - .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema(INDEX_SCHEMA) .rows(ROWS1) .buildMMappedIndex(); final QueryableIndex index2 = IndexBuilder.create() .tmpDir(new File(tmpDir, "2")) - .outputMediumFactory(OffHeapMemoryOutputMediumFactory.instance()) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .schema(INDEX_SCHEMA) .rows(ROWS2) .buildMMappedIndex(); From 832ca66c888ac7a2651ea22ccbb796621ff60648 Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 7 Nov 2017 01:50:32 -0300 Subject: [PATCH 13/14] Add comments to ByteBufferInputStream --- .../src/main/java/io/druid/io/ByteBufferInputStream.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/common/src/main/java/io/druid/io/ByteBufferInputStream.java b/common/src/main/java/io/druid/io/ByteBufferInputStream.java index 3d9f732f47b6..189bf777e512 100644 --- a/common/src/main/java/io/druid/io/ByteBufferInputStream.java +++ b/common/src/main/java/io/druid/io/ByteBufferInputStream.java @@ -23,10 +23,17 @@ 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; From 66378a1bd90f5013872094f7b480249b1b3d0f19 Mon Sep 17 00:00:00 2001 From: leventov Date: Mon, 20 Nov 2017 21:07:42 -0300 Subject: [PATCH 14/14] Remove unused declarations --- .../CompressedVSizeIndexedBenchmark.java | 3 +-- .../benchmark/LongCompressionBenchmark.java | 2 +- .../segment/CompressedVSizeIndexedSupplier.java | 3 +-- .../CompressedVSizeIndexedV3Supplier.java | 7 +------ .../src/main/java/io/druid/segment/IndexIO.java | 3 +-- .../data/CompressedIntsIndexedWriter.java | 16 ---------------- .../data/CompressedLongsIndexedSupplier.java | 7 +------ .../druid/segment/data/CompressionFactory.java | 2 -- .../druid/segment/data/CompressionStrategy.java | 3 --- .../segment/data/DeltaLongEncodingReader.java | 6 ------ .../segment/data/LongsLongEncodingReader.java | 8 -------- .../segment/data/TableLongEncodingReader.java | 6 ------ .../io/druid/segment/data/VSizeIndexedInts.java | 6 ------ .../serde/DictionaryEncodedColumnPartSerde.java | 9 ++++----- .../serde/LongGenericColumnPartSerde.java | 3 +-- .../CompressedVSizeIndexedV3SupplierTest.java | 3 +-- .../segment/data/CompressedLongsSerdeTest.java | 5 ++--- .../data/CompressedVSizeIndexedSupplierTest.java | 3 +-- .../data/CompressedVSizeIndexedV3WriterTest.java | 6 ++---- 19 files changed, 17 insertions(+), 84 deletions(-) diff --git a/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java index da42f154e8da..f829bccf9399 100644 --- a/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java +++ b/benchmarks/src/main/java/io/druid/benchmark/CompressedVSizeIndexedBenchmark.java @@ -103,8 +103,7 @@ public IndexedInts apply(int[] input) ); this.compressed = CompressedVSizeIndexedSupplier.fromByteBuffer( bufferCompressed, - ByteOrder.nativeOrder(), - null + ByteOrder.nativeOrder() ).get(); final ByteBuffer bufferUncompressed = serialize( diff --git a/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java b/benchmarks/src/main/java/io/druid/benchmark/LongCompressionBenchmark.java index ac41c6874571..cfb40ceabd5f 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 void setup() throws Exception 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/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java index d98d16da29ee..45043e18a47c 100644 --- a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedSupplier.java @@ -24,7 +24,6 @@ import io.druid.java.util.common.IAE; 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.query.monomorphicprocessing.RuntimeShapeInspector; import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; import io.druid.segment.data.CompressionStrategy; @@ -82,7 +81,7 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I valueSupplier.writeTo(channel, smoosher); } - public static CompressedVSizeIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order, SmooshedFileMapper fileMapper) + public static CompressedVSizeIndexedSupplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); diff --git a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java index 1c1ebd080164..6d4f0330942e 100644 --- a/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java +++ b/processing/src/main/java/io/druid/segment/CompressedVSizeIndexedV3Supplier.java @@ -23,7 +23,6 @@ import io.druid.java.util.common.IAE; 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.data.CompressedIntsIndexedSupplier; import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier; import io.druid.segment.data.CompressionStrategy; @@ -61,11 +60,7 @@ private CompressedVSizeIndexedV3Supplier( this.valueSupplier = valueSupplier; } - public static CompressedVSizeIndexedV3Supplier fromByteBuffer( - ByteBuffer buffer, - ByteOrder order, - SmooshedFileMapper fileMapper - ) + public static CompressedVSizeIndexedV3Supplier fromByteBuffer(ByteBuffer buffer, ByteOrder order) { byte versionFromBuffer = buffer.get(); diff --git a/processing/src/main/java/io/druid/segment/IndexIO.java b/processing/src/main/java/io/druid/segment/IndexIO.java index 46e9faa75b62..60faadd177c2 100644 --- a/processing/src/main/java/io/druid/segment/IndexIO.java +++ b/processing/src/main/java/io/druid/segment/IndexIO.java @@ -349,8 +349,7 @@ public MMappedIndex mapDir(File inDir) throws IOException 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/data/CompressedIntsIndexedWriter.java b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java index 0dd0e1f456fa..2feacc73d05a 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedIntsIndexedWriter.java @@ -21,7 +21,6 @@ import io.druid.java.util.common.io.Closer; import io.druid.java.util.common.io.smoosh.FileSmoosher; -import io.druid.segment.IndexIO; import io.druid.segment.serde.MetaSerdeHelper; import io.druid.segment.writeout.SegmentWriteOutMedium; @@ -43,21 +42,6 @@ public class CompressedIntsIndexedWriter extends SingleValueIndexedIntsWriter .writeInt(x -> x.chunkFactor) .writeByte(x -> x.compression.getId()); - public static CompressedIntsIndexedWriter create( - final SegmentWriteOutMedium segmentWriteOutMedium, - final String filenameBase, - final CompressionStrategy compression - ) - { - return new CompressedIntsIndexedWriter( - segmentWriteOutMedium, - filenameBase, - CompressedIntsIndexedSupplier.MAX_INTS_IN_BUFFER, - IndexIO.BYTE_ORDER, - compression - ); - } - private final int chunkFactor; private final CompressionStrategy compression; private final GenericIndexedWriter flattener; 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 c7428b538035..b408b54de6ae 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java +++ b/processing/src/main/java/io/druid/segment/data/CompressedLongsIndexedSupplier.java @@ -23,7 +23,6 @@ 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.serde.MetaSerdeHelper; import io.druid.segment.serde.Serializer; @@ -98,11 +97,7 @@ public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws I 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(); 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 a9b73ca1bb55..1e6a2ea8a007 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionFactory.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionFactory.java @@ -283,8 +283,6 @@ public interface LongEncodingReader long read(int index); - int getNumBytes(int values); - LongEncodingReader duplicate(); } diff --git a/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java index 0025128a4bbc..c4a6fd742be5 100644 --- a/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java +++ b/processing/src/main/java/io/druid/segment/data/CompressionStrategy.java @@ -29,7 +29,6 @@ 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.java.util.common.logger.Logger; import io.druid.segment.CompressedPools; import net.jpountz.lz4.LZ4Factory; import net.jpountz.lz4.LZ4SafeDecompressor; @@ -107,8 +106,6 @@ public Compressor getCompressor() }; public static final CompressionStrategy DEFAULT_COMPRESSION_STRATEGY = LZ4; - private static final Logger log = new Logger(CompressionStrategy.class); - final byte id; CompressionStrategy(byte id) 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 5c21615c9141..a04b1b9b62fb 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 long read(int index) return base + deserializer.get(index); } - @Override - public int getNumBytes(int values) - { - return VSizeLongSerde.getSerializedSize(bitsPerValue, values); - } - @Override public CompressionFactory.LongEncodingReader duplicate() { 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 9cbb4f96a70b..078226e1cfc8 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 long read(int index) return buffer.get(buffer.position() + index); } - @Override - public int getNumBytes(int values) - { - return values * Longs.BYTES; - } - @Override public CompressionFactory.LongEncodingReader duplicate() { 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 db79c23229b3..bd1c10e029fc 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 long read(int index) return table[(int) deserializer.get(index)]; } - @Override - public int getNumBytes(int values) - { - return VSizeLongSerde.getSerializedSize(bitsPerValue, values); - } - @Override public CompressionFactory.LongEncodingReader duplicate() { 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 ae5bf28b22dc..1c499040bb00 100644 --- a/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java +++ b/processing/src/main/java/io/druid/segment/data/VSizeIndexedInts.java @@ -29,7 +29,6 @@ import io.druid.segment.writeout.HeapByteBufferWriteOutBytes; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntList; -import it.unimi.dsi.fastutil.ints.IntLists; import java.io.IOException; import java.nio.ByteBuffer; @@ -56,11 +55,6 @@ public static VSizeIndexedInts fromArray(int[] array, int maxValue) return fromList(IntArrayList.wrap(array), maxValue); } - public static VSizeIndexedInts empty() - { - return fromList(IntLists.EMPTY_LIST, 0); - } - public static VSizeIndexedInts fromList(IntList list, int maxValue) { int numBytes = getNumBytesForMax(maxValue); 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 7cbc55e1bac0..c77078efb41f 100644 --- a/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/DictionaryEncodedColumnPartSerde.java @@ -28,7 +28,6 @@ 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; @@ -296,7 +295,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo final WritableSupplier> rMultiValuedColumn; if (hasMultipleValues) { - rMultiValuedColumn = readMultiValuedColumn(rVersion, buffer, rFlags, builder.getFileMapper()); + rMultiValuedColumn = readMultiValuedColumn(rVersion, buffer, rFlags); rSingleValuedColumn = null; } else { rSingleValuedColumn = readSingleValuedColumn(rVersion, buffer); @@ -346,7 +345,7 @@ private WritableSupplier readSingleValuedColumn(VERSION version, By } private WritableSupplier> readMultiValuedColumn( - VERSION version, ByteBuffer buffer, int flags, SmooshedFileMapper fileMapper + VERSION version, ByteBuffer buffer, int flags ) { switch (version) { @@ -355,9 +354,9 @@ private WritableSupplier> readMultiValuedColumn( } 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/LongGenericColumnPartSerde.java b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java index e33ef1c974f2..758bb07fa30e 100644 --- a/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java +++ b/processing/src/main/java/io/druid/segment/serde/LongGenericColumnPartSerde.java @@ -101,8 +101,7 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo { final CompressedLongsIndexedSupplier column = CompressedLongsIndexedSupplier.fromByteBuffer( buffer, - byteOrder, - builder.getFileMapper() + byteOrder ); builder.setType(ValueType.LONG) .setHasMultipleValues(false) diff --git a/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java index b87787aac5a0..e3538fd70e20 100644 --- a/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java +++ b/processing/src/test/java/io/druid/segment/CompressedVSizeIndexedV3SupplierTest.java @@ -83,8 +83,7 @@ protected WritableSupplier> fromByteBuffer(ByteBu { return CompressedVSizeIndexedV3Supplier.fromByteBuffer( buffer, - ByteOrder.nativeOrder(), - null + ByteOrder.nativeOrder() ); } } 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 a628de90f5f6..625b0c2f8182 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedLongsSerdeTest.java @@ -147,7 +147,7 @@ public void testValues(long[] values) throws Exception 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,8 +203,7 @@ private void testSupplierSerde(CompressedLongsIndexedSupplier supplier, long[] v 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 f20bf15dfbd0..9a98be6c10e9 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedSupplierTest.java @@ -146,8 +146,7 @@ protected WritableSupplier> fromByteBuffer(ByteBu { 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 46fe2f4bb491..b3f86aef9211 100644 --- a/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java +++ b/processing/src/test/java/io/druid/segment/data/CompressedVSizeIndexedV3WriterTest.java @@ -147,8 +147,7 @@ private void checkSerializedSizeAndData(int offsetChunkFactor, int valueChunkFac // read from ByteBuffer and check values CompressedVSizeIndexedV3Supplier supplierFromByteBuffer = CompressedVSizeIndexedV3Supplier.fromByteBuffer( ByteBuffer.wrap(IOUtils.toByteArray(writeOutBytes.asInputStream())), - byteOrder, - null + byteOrder ); try (final IndexedMultivalue indexedMultivalue = supplierFromByteBuffer.get()) { @@ -272,8 +271,7 @@ private void checkV2SerializedSizeAndData(int offsetChunkFactor, int valueChunkF CompressedVSizeIndexedV3Supplier supplierFromByteBuffer = CompressedVSizeIndexedV3Supplier.fromByteBuffer( mapper.mapFile("test"), - byteOrder, - mapper + byteOrder ); IndexedMultivalue indexedMultivalue = supplierFromByteBuffer.get(); assertEquals(indexedMultivalue.size(), vals.size());