Replace IOPeon with SegmentWriteOutMedium; Improve buffer compression (#4762)

* Replace IOPeon with OutputMedium; Improve compression

* Fix test

* Cleanup CompressionStrategy

* Javadocs

* Add OutputBytesTest

* Address comments

* Random access in OutputBytes and GenericIndexedWriter

* Fix bugs

* Fixes

* Test OutputBytes.readFully()

* Address comments

* Rename OutputMedium to SegmentWriteOutMedium and OutputBytes to WriteOutBytes

* Add comments to ByteBufferInputStream

* Remove unused declarations
This commit is contained in:
Roman Leventov 2017-12-04 23:04:27 -03:00 committed by Slim
parent 7c01f77b04
commit a7a6a0487e
235 changed files with 5332 additions and 4531 deletions

View File

@ -19,12 +19,13 @@
package io.druid.benchmark;
import com.google.common.primitives.Ints;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.java.util.common.io.Closer;
import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.VSizeIndexedInts;
import io.druid.segment.data.WritableSupplier;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Mode;
@ -69,25 +70,20 @@ public class CompressedIndexedIntsBenchmark
}
final ByteBuffer bufferCompressed = serialize(
CompressedVSizeIntsIndexedSupplier.fromList(
Ints.asList(vals),
IntArrayList.wrap(vals),
bound - 1,
CompressedVSizeIntsIndexedSupplier.maxIntsInBufferForBytes(bytes),
ByteOrder.nativeOrder(), CompressedObjectStrategy.CompressionStrategy.LZ4
ByteOrder.nativeOrder(),
CompressionStrategy.LZ4,
Closer.create()
)
);
this.compressed = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(
bufferCompressed,
ByteOrder.nativeOrder(),
null
ByteOrder.nativeOrder()
).get();
final ByteBuffer bufferUncompressed = serialize(
new VSizeIndexedInts.VSizeIndexedIntsSupplier(
VSizeIndexedInts.fromArray(
vals
)
)
);
final ByteBuffer bufferUncompressed = serialize(VSizeIndexedInts.fromArray(vals));
this.uncompressed = VSizeIndexedInts.readFromByteBuffer(bufferUncompressed);
filter = new BitSet();
@ -128,7 +124,7 @@ public class CompressedIndexedIntsBenchmark
}
};
writableSupplier.writeToChannel(channel);
writableSupplier.writeTo(channel, null);
buffer.rewind();
return buffer;
}

View File

@ -22,8 +22,9 @@ package io.druid.benchmark;
import com.google.common.base.Function;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import io.druid.java.util.common.io.Closer;
import io.druid.segment.CompressedVSizeIndexedSupplier;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.IndexedMultivalue;
import io.druid.segment.data.VSizeIndexed;
@ -95,13 +96,14 @@ public class CompressedVSizeIndexedBenchmark
}
),
bound - 1,
ByteOrder.nativeOrder(), CompressedObjectStrategy.CompressionStrategy.LZ4
ByteOrder.nativeOrder(),
CompressionStrategy.LZ4,
Closer.create()
)
);
this.compressed = CompressedVSizeIndexedSupplier.fromByteBuffer(
bufferCompressed,
ByteOrder.nativeOrder(),
null
ByteOrder.nativeOrder()
).get();
final ByteBuffer bufferUncompressed = serialize(
@ -117,7 +119,7 @@ public class CompressedVSizeIndexedBenchmark
}
}
)
).asWritableSupplier()
)
);
this.uncompressed = VSizeIndexed.readFromByteBuffer(bufferUncompressed);
@ -159,7 +161,7 @@ public class CompressedVSizeIndexedBenchmark
}
};
writableSupplier.writeToChannel(channel);
writableSupplier.writeTo(channel, null);
buffer.rewind();
return buffer;
}

View File

@ -36,6 +36,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.js.JavaScriptConfig;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.extraction.ExtractionFn;
@ -132,6 +133,7 @@ public class FilterPartitionBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -141,7 +143,7 @@ public class FilterPartitionBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
@Setup
@ -178,7 +180,8 @@ public class FilterPartitionBenchmark
indexFile = INDEX_MERGER_V9.persist(
incIndex,
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
qIndex = INDEX_IO.loadIndex(indexFile);

View File

@ -35,6 +35,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.js.JavaScriptConfig;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
@ -128,6 +129,7 @@ public class FilteredAggregatorBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -137,7 +139,7 @@ public class FilteredAggregatorBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
@Setup
@ -194,7 +196,8 @@ public class FilteredAggregatorBenchmark
indexFile = INDEX_MERGER_V9.persist(
incIndex,
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
qIndex = INDEX_IO.loadIndex(indexFile);

View File

@ -72,7 +72,7 @@ public class FloatCompressionBenchmark
File compFile = new File(dir, file + "-" + strategy);
rand = new Random();
ByteBuffer buffer = Files.map(compFile);
supplier = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder(), null);
supplier = CompressedFloatsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder());
}
@Benchmark

View File

@ -20,24 +20,20 @@
package io.druid.benchmark;
import com.google.common.collect.ImmutableList;
import com.google.common.io.ByteSink;
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.FloatSupplierSerializer;
import io.druid.segment.data.TmpFileIOPeon;
import java.io.BufferedReader;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.io.Writer;
import java.net.URISyntaxException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.FileChannel;
import java.nio.charset.StandardCharsets;
@ -51,10 +47,10 @@ public class FloatCompressionBenchmarkFileGenerator
{
private static final Logger log = new Logger(FloatCompressionBenchmarkFileGenerator.class);
public static final int ROW_NUM = 5000000;
public static final List<CompressedObjectStrategy.CompressionStrategy> compressions =
public static final List<CompressionStrategy> compressions =
ImmutableList.of(
CompressedObjectStrategy.CompressionStrategy.LZ4,
CompressedObjectStrategy.CompressionStrategy.NONE
CompressionStrategy.LZ4,
CompressionStrategy.NONE
);
private static String dirPath = "floatCompress/";
@ -143,48 +139,30 @@ public class FloatCompressionBenchmarkFileGenerator
// create compressed files using all combinations of CompressionStrategy and FloatEncoding provided
for (Map.Entry<String, BenchmarkColumnValueGenerator> entry : generators.entrySet()) {
for (CompressedObjectStrategy.CompressionStrategy compression : compressions) {
for (CompressionStrategy compression : compressions) {
String name = entry.getKey() + "-" + compression.toString();
log.info("%s: ", name);
File compFile = new File(dir, name);
compFile.delete();
File dataFile = new File(dir, entry.getKey());
TmpFileIOPeon iopeon = new TmpFileIOPeon(true);
FloatSupplierSerializer writer = CompressionFactory.getFloatSerializer(
iopeon,
new OffHeapMemorySegmentWriteOutMedium(),
"float",
ByteOrder.nativeOrder(),
compression
);
BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8);
try (FileChannel output = FileChannel.open(
compFile.toPath(),
StandardOpenOption.CREATE_NEW,
StandardOpenOption.WRITE
)) {
try (
BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8);
FileChannel output =
FileChannel.open(compFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)
) {
writer.open();
String line;
while ((line = br.readLine()) != null) {
writer.add(Float.parseFloat(line));
}
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
writer.closeAndConsolidate(
new ByteSink()
{
@Override
public OutputStream openStream() throws IOException
{
return baos;
}
}
);
output.write(ByteBuffer.wrap(baos.toByteArray()));
}
finally {
iopeon.close();
br.close();
writer.writeTo(output, null);
}
log.info("%d", compFile.length() / 1024);
}

View File

@ -23,10 +23,10 @@ import com.google.common.io.Files;
import com.google.common.primitives.Ints;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.GenericIndexedWriter;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.data.TmpFileIOPeon;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
@ -106,7 +106,7 @@ public class GenericIndexedBenchmark
public void createGenericIndexed() throws IOException
{
GenericIndexedWriter<byte[]> genericIndexedWriter = new GenericIndexedWriter<>(
new TmpFileIOPeon(),
new OffHeapMemorySegmentWriteOutMedium(),
"genericIndexedBenchmark",
byteArrayStrategy
);
@ -121,14 +121,13 @@ public class GenericIndexedBenchmark
element.putInt(0, i);
genericIndexedWriter.write(element.array());
}
genericIndexedWriter.close();
smooshDir = Files.createTempDir();
file = File.createTempFile("genericIndexedBenchmark", "meta");
try (FileChannel fileChannel =
FileChannel.open(file.toPath(), StandardOpenOption.CREATE, StandardOpenOption.WRITE);
FileSmoosher fileSmoosher = new FileSmoosher(smooshDir)) {
genericIndexedWriter.writeToChannel(fileChannel, fileSmoosher);
genericIndexedWriter.writeTo(fileChannel, fileSmoosher);
}
FileChannel fileChannel = FileChannel.open(file.toPath());

View File

@ -46,6 +46,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.DruidProcessingConfig;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
@ -153,6 +154,7 @@ public class GroupByTypeInterfaceBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -162,7 +164,7 @@ public class GroupByTypeInterfaceBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, GroupByQuery>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
@ -339,7 +341,8 @@ public class GroupByTypeInterfaceBenchmark
final File file = INDEX_MERGER_V9.persist(
index,
new File(tmpDir, String.valueOf(i)),
new IndexSpec()
new IndexSpec(),
null
);
queryableIndexes.add(INDEX_IO.loadIndex(file));

View File

@ -75,7 +75,7 @@ public class LongCompressionBenchmark
File compFile = new File(dir, file + "-" + strategy + "-" + format);
rand = new Random();
ByteBuffer buffer = Files.map(compFile);
supplier = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder(), null);
supplier = CompressedLongsIndexedSupplier.fromByteBuffer(buffer, ByteOrder.nativeOrder());
}
@Benchmark

View File

@ -20,24 +20,20 @@
package io.druid.benchmark;
import com.google.common.collect.ImmutableList;
import com.google.common.io.ByteSink;
import io.druid.benchmark.datagen.BenchmarkColumnSchema;
import io.druid.benchmark.datagen.BenchmarkColumnValueGenerator;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.LongSupplierSerializer;
import io.druid.segment.data.TmpFileIOPeon;
import java.io.BufferedReader;
import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
import java.io.OutputStream;
import java.io.Writer;
import java.net.URISyntaxException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.FileChannel;
import java.nio.charset.StandardCharsets;
@ -51,9 +47,10 @@ public class LongCompressionBenchmarkFileGenerator
{
private static final Logger log = new Logger(LongCompressionBenchmarkFileGenerator.class);
public static final int ROW_NUM = 5000000;
public static final List<CompressedObjectStrategy.CompressionStrategy> compressions =
ImmutableList.of(CompressedObjectStrategy.CompressionStrategy.LZ4,
CompressedObjectStrategy.CompressionStrategy.NONE);
public static final List<CompressionStrategy> compressions =
ImmutableList.of(
CompressionStrategy.LZ4,
CompressionStrategy.NONE);
public static final List<CompressionFactory.LongEncodingStrategy> encodings =
ImmutableList.of(CompressionFactory.LongEncodingStrategy.AUTO, CompressionFactory.LongEncodingStrategy.LONGS);
@ -134,7 +131,7 @@ public class LongCompressionBenchmarkFileGenerator
// create compressed files using all combinations of CompressionStrategy and LongEncoding provided
for (Map.Entry<String, BenchmarkColumnValueGenerator> entry : generators.entrySet()) {
for (CompressedObjectStrategy.CompressionStrategy compression : compressions) {
for (CompressionStrategy compression : compressions) {
for (CompressionFactory.LongEncodingStrategy encoding : encodings) {
String name = entry.getKey() + "-" + compression.toString() + "-" + encoding.toString();
log.info("%s: ", name);
@ -142,42 +139,24 @@ public class LongCompressionBenchmarkFileGenerator
compFile.delete();
File dataFile = new File(dir, entry.getKey());
TmpFileIOPeon iopeon = new TmpFileIOPeon(true);
LongSupplierSerializer writer = CompressionFactory.getLongSerializer(
iopeon,
new OffHeapMemorySegmentWriteOutMedium(),
"long",
ByteOrder.nativeOrder(),
encoding,
compression
);
BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8);
try (FileChannel output = FileChannel.open(
compFile.toPath(),
StandardOpenOption.CREATE_NEW,
StandardOpenOption.WRITE
)) {
try (
BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8);
FileChannel output =
FileChannel.open(compFile.toPath(), StandardOpenOption.CREATE_NEW, StandardOpenOption.WRITE)
) {
writer.open();
String line;
while ((line = br.readLine()) != null) {
writer.add(Long.parseLong(line));
}
final ByteArrayOutputStream baos = new ByteArrayOutputStream();
writer.closeAndConsolidate(
new ByteSink()
{
@Override
public OutputStream openStream() throws IOException
{
return baos;
}
}
);
output.write(ByteBuffer.wrap(baos.toByteArray()));
}
finally {
iopeon.close();
br.close();
writer.writeTo(output, null);
}
log.info("%d", compFile.length() / 1024);
}

View File

@ -37,6 +37,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
@ -136,6 +137,7 @@ public class TopNTypeInterfaceBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -145,7 +147,7 @@ public class TopNTypeInterfaceBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, TopNQueryBuilder>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
@ -291,7 +293,8 @@ public class TopNTypeInterfaceBenchmark
File indexFile = INDEX_MERGER_V9.persist(
incIndexes.get(i),
tmpFile,
new IndexSpec()
new IndexSpec(),
null
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);

View File

@ -33,6 +33,7 @@ import io.druid.hll.HyperLogLogHash;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.segment.IndexBuilder;
@ -144,8 +145,8 @@ public class SegmentGenerator implements Closeable
return Iterables.getOnlyElement(indexes);
} else {
try {
final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex(
TestHelper.getTestIndexMergerV9().merge(
final QueryableIndex merged = TestHelper.getTestIndexIO(OffHeapMemorySegmentWriteOutMediumFactory.instance()).loadIndex(
TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()).merge(
indexes.stream().map(QueryableIndexIndexableAdapter::new).collect(Collectors.toList()),
false,
schemaInfo.getAggs()
@ -186,7 +187,7 @@ public class SegmentGenerator implements Closeable
.create()
.schema(indexSchema)
.tmpDir(new File(new File(tempDir, identifier), String.valueOf(indexNumber)))
.indexMerger(TestHelper.getTestIndexMergerV9())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.rows(rows)
.buildMMappedIndex();
}

View File

@ -39,6 +39,7 @@ import io.druid.segment.column.ColumnConfig;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.serde.ComplexMetrics;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.commons.io.FileUtils;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@ -95,6 +96,7 @@ public class IndexMergeBenchmark
JSON_MAPPER.setInjectableValues(injectableValues);
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -104,7 +106,7 @@ public class IndexMergeBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
@Setup
@ -144,7 +146,8 @@ public class IndexMergeBenchmark
File indexFile = INDEX_MERGER_V9.persist(
incIndex,
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);
@ -188,7 +191,8 @@ public class IndexMergeBenchmark
rollup,
schemaInfo.getAggsArray(),
tmpFile,
new IndexSpec()
new IndexSpec(),
null
);
blackhole.consume(mergedFile);

View File

@ -28,6 +28,7 @@ import io.druid.data.input.InputRow;
import io.druid.hll.HyperLogLogHash;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexMergerV9;
@ -88,6 +89,7 @@ public class IndexPersistBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -97,7 +99,7 @@ public class IndexPersistBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
@Setup
@ -172,7 +174,8 @@ public class IndexPersistBenchmark
File indexFile = INDEX_MERGER_V9.persist(
incIndex,
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
blackhole.consume(indexFile);

View File

@ -47,6 +47,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.DruidProcessingConfig;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
@ -158,6 +159,7 @@ public class GroupByBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -167,7 +169,7 @@ public class GroupByBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, GroupByQuery>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
@ -434,7 +436,8 @@ public class GroupByBenchmark
final File file = INDEX_MERGER_V9.persist(
index,
new File(tmpDir, String.valueOf(i)),
new IndexSpec()
new IndexSpec(),
null
);
queryableIndexes.add(INDEX_IO.loadIndex(file));

View File

@ -38,6 +38,7 @@ import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.Druids;
import io.druid.query.Druids.SearchQueryBuilder;
import io.druid.query.FinalizeResultsQueryRunner;
@ -140,6 +141,7 @@ public class SearchBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -149,7 +151,7 @@ public class SearchBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, Druids.SearchQueryBuilder>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
@ -360,7 +362,8 @@ public class SearchBenchmark
File indexFile = INDEX_MERGER_V9.persist(
incIndexes.get(i),
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);

View File

@ -37,6 +37,7 @@ import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
@ -134,6 +135,7 @@ public class SelectBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -143,7 +145,7 @@ public class SelectBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, Druids.SelectQueryBuilder>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
@ -223,7 +225,8 @@ public class SelectBenchmark
File indexFile = INDEX_MERGER_V9.persist(
incIndexes.get(i),
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);
qIndexes.add(qIndex);

View File

@ -35,6 +35,7 @@ import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.Druids;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
@ -132,6 +133,7 @@ public class TimeseriesBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -141,7 +143,7 @@ public class TimeseriesBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, TimeseriesQuery>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
@ -286,7 +288,8 @@ public class TimeseriesBenchmark
File indexFile = INDEX_MERGER_V9.persist(
incIndexes.get(i),
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);

View File

@ -36,6 +36,7 @@ import io.druid.java.util.common.guava.Sequence;
import io.druid.java.util.common.guava.Sequences;
import io.druid.java.util.common.logger.Logger;
import io.druid.offheap.OffheapBufferGenerator;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.FinalizeResultsQueryRunner;
import io.druid.query.Query;
import io.druid.query.QueryPlus;
@ -133,6 +134,7 @@ public class TopNBenchmark
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -142,7 +144,7 @@ public class TopNBenchmark
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
}
private static final Map<String, Map<String, TopNQueryBuilder>> SCHEMA_QUERY_MAP = new LinkedHashMap<>();
@ -263,7 +265,8 @@ public class TopNBenchmark
File indexFile = INDEX_MERGER_V9.persist(
incIndexes.get(i),
tmpDir,
new IndexSpec()
new IndexSpec(),
null
);
QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile);

View File

@ -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() {}
}

View File

@ -24,6 +24,7 @@ import com.google.common.io.OutputSupplier;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;
import io.druid.io.Channels;
import io.druid.java.util.common.StringUtils;
import java.io.IOException;
@ -38,38 +39,6 @@ import java.util.List;
public class SerializerUtils
{
/**
* Writes the given long value into the given OutputStream in big-endian byte order, using the helperBuffer. Faster
* alternative to out.write(Longs.toByteArray(value)), more convenient (sometimes) than wrapping the OutputStream into
* {@link java.io.DataOutputStream}.
*
* @param helperBuffer a big-endian heap ByteBuffer with capacity of at least 8
*/
public static void writeBigEndianLongToOutputStream(OutputStream out, long value, ByteBuffer helperBuffer)
throws IOException
{
if (helperBuffer.order() != ByteOrder.BIG_ENDIAN || !helperBuffer.hasArray()) {
throw new IllegalArgumentException("Expected writable, big-endian, heap byteBuffer");
}
helperBuffer.putLong(0, value);
out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Longs.BYTES);
}
/**
* Writes the given long value into the given OutputStream in the native byte order, using the helperBuffer.
*
* @param helperBuffer a heap ByteBuffer with capacity of at least 8, with the native byte order
*/
public static void writeNativeOrderedLongToOutputStream(OutputStream out, long value, ByteBuffer helperBuffer)
throws IOException
{
if (helperBuffer.order() != ByteOrder.nativeOrder() || !helperBuffer.hasArray()) {
throw new IllegalArgumentException("Expected writable heap byteBuffer with the native byte order");
}
helperBuffer.putLong(0, value);
out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Longs.BYTES);
}
/**
* Writes the given int value into the given OutputStream in big-endian byte order, using the helperBuffer. Faster
* alternative to out.write(Ints.toByteArray(value)), more convenient (sometimes) than wrapping the OutputStream into
@ -87,21 +56,6 @@ public class SerializerUtils
out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Ints.BYTES);
}
/**
* Writes the given int value into the given OutputStream in the native byte order, using the given helperBuffer.
*
* @param helperBuffer a heap ByteBuffer with capacity of at least 4, with the native byte order
*/
public static void writeNativeOrderedIntToOutputStream(OutputStream out, int value, ByteBuffer helperBuffer)
throws IOException
{
if (helperBuffer.order() != ByteOrder.nativeOrder() || !helperBuffer.hasArray()) {
throw new IllegalArgumentException("Expected writable heap byteBuffer with the native byte order");
}
helperBuffer.putInt(0, value);
out.write(helperBuffer.array(), helperBuffer.arrayOffset(), Ints.BYTES);
}
public <T extends OutputStream> void writeString(T out, String name) throws IOException
{
byte[] nameBytes = StringUtils.toUtf8(name);
@ -120,10 +74,10 @@ public class SerializerUtils
{
byte[] nameBytes = StringUtils.toUtf8(name);
writeInt(out, nameBytes.length);
out.write(ByteBuffer.wrap(nameBytes));
Channels.writeFully(out, ByteBuffer.wrap(nameBytes));
}
public String readString(InputStream in) throws IOException
String readString(InputStream in) throws IOException
{
final int length = readInt(in);
byte[] stringBytes = new byte[length];
@ -144,12 +98,12 @@ public class SerializerUtils
return bytes;
}
public void writeStrings(OutputStream out, String[] names) throws IOException
void writeStrings(OutputStream out, String[] names) throws IOException
{
writeStrings(out, Arrays.asList(names));
}
public void writeStrings(OutputStream out, List<String> names) throws IOException
private void writeStrings(OutputStream out, List<String> names) throws IOException
{
writeInt(out, names.size());
@ -158,7 +112,7 @@ public class SerializerUtils
}
}
public String[] readStrings(InputStream in) throws IOException
String[] readStrings(InputStream in) throws IOException
{
int length = readInt(in);
@ -171,7 +125,7 @@ public class SerializerUtils
return retVal;
}
public String[] readStrings(ByteBuffer in) throws IOException
String[] readStrings(ByteBuffer in) throws IOException
{
int length = in.getInt();
@ -184,20 +138,20 @@ public class SerializerUtils
return retVal;
}
public void writeInt(OutputStream out, int intValue) throws IOException
private void writeInt(OutputStream out, int intValue) throws IOException
{
out.write(Ints.toByteArray(intValue));
}
public void writeInt(WritableByteChannel out, int intValue) throws IOException
private void writeInt(WritableByteChannel out, int intValue) throws IOException
{
final ByteBuffer buffer = ByteBuffer.allocate(Ints.BYTES);
buffer.putInt(intValue);
buffer.flip();
out.write(buffer);
Channels.writeFully(out, buffer);
}
public int readInt(InputStream in) throws IOException
private int readInt(InputStream in) throws IOException
{
byte[] intBytes = new byte[Ints.BYTES];
@ -206,7 +160,7 @@ public class SerializerUtils
return Ints.fromByteArray(intBytes);
}
public void writeInts(OutputStream out, int[] ints) throws IOException
void writeInts(OutputStream out, int[] ints) throws IOException
{
writeInt(out, ints.length);
@ -215,7 +169,7 @@ public class SerializerUtils
}
}
public int[] readInts(InputStream in) throws IOException
int[] readInts(InputStream in) throws IOException
{
int size = readInt(in);
@ -227,7 +181,7 @@ public class SerializerUtils
return retVal;
}
public void writeLong(OutputStream out, long longValue) throws IOException
private void writeLong(OutputStream out, long longValue) throws IOException
{
out.write(Longs.toByteArray(longValue));
}
@ -237,10 +191,10 @@ public class SerializerUtils
final ByteBuffer buffer = ByteBuffer.allocate(Longs.BYTES);
buffer.putLong(longValue);
buffer.flip();
out.write(buffer);
Channels.writeFully(out, buffer);
}
public long readLong(InputStream in) throws IOException
long readLong(InputStream in) throws IOException
{
byte[] longBytes = new byte[Longs.BYTES];
@ -249,7 +203,7 @@ public class SerializerUtils
return Longs.fromByteArray(longBytes);
}
public void writeLongs(OutputStream out, long[] longs) throws IOException
void writeLongs(OutputStream out, long[] longs) throws IOException
{
writeInt(out, longs.length);
@ -258,7 +212,7 @@ public class SerializerUtils
}
}
public long[] readLongs(InputStream in) throws IOException
long[] readLongs(InputStream in) throws IOException
{
int size = readInt(in);
@ -275,20 +229,20 @@ public class SerializerUtils
writeInt(out, Float.floatToRawIntBits(floatValue));
}
public void writeFloat(WritableByteChannel out, float floatValue) throws IOException
void writeFloat(WritableByteChannel out, float floatValue) throws IOException
{
final ByteBuffer buffer = ByteBuffer.allocate(Floats.BYTES);
buffer.putFloat(floatValue);
buffer.flip();
out.write(buffer);
Channels.writeFully(out, buffer);
}
public float readFloat(InputStream in) throws IOException
float readFloat(InputStream in) throws IOException
{
return Float.intBitsToFloat(readInt(in));
}
public void writeFloats(OutputStream out, float[] floats) throws IOException
void writeFloats(OutputStream out, float[] floats) throws IOException
{
writeInt(out, floats.length);
@ -297,7 +251,7 @@ public class SerializerUtils
}
}
public float[] readFloats(InputStream in) throws IOException
float[] readFloats(InputStream in) throws IOException
{
int size = readInt(in);

View File

@ -0,0 +1,71 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.io;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
/**
* Streams bytes from the {@link ByteBuffer}'s position to the limit as {@link InputStream}.
*/
public final class ByteBufferInputStream extends InputStream
{
private final ByteBuffer buffer;
/**
* Does *not* make a copy of the given buffer, so the position of the given buffer is incremented, as the created
* InputStream is used.
*/
public ByteBufferInputStream(ByteBuffer buffer)
{
this.buffer = buffer;
}
@Override
public int read()
{
if (!buffer.hasRemaining()) {
return -1;
}
return buffer.get() & 0xFF;
}
@Override
public int read(byte[] bytes, int off, int len)
{
if (len == 0) {
return 0;
}
if (!buffer.hasRemaining()) {
return -1;
}
len = Math.min(len, buffer.remaining());
buffer.get(bytes, off, len);
return len;
}
@Override
public int available() throws IOException
{
return buffer.remaining();
}
}

View File

@ -17,19 +17,23 @@
* under the License.
*/
package io.druid.segment.data;
package io.druid.io;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;
/**
*/
public interface IOPeon extends Closeable
public final class Channels
{
OutputStream makeOutputStream(String filename) throws IOException;
InputStream makeInputStream(String filename) throws IOException;
File getFile(String filename);
public static void writeFully(WritableByteChannel channel, ByteBuffer src) throws IOException
{
while (src.remaining() > 0) {
channel.write(src);
}
}
private Channels()
{
}
}

View File

@ -343,3 +343,26 @@ If the peon is running in remote mode, there must be an overlord up and running.
|`druid.peon.taskActionClient.retry.minWait`|The minimum retry time to communicate with overlord.|PT5S|
|`druid.peon.taskActionClient.retry.maxWait`|The maximum retry time to communicate with overlord.|PT1M|
|`druid.peon.taskActionClient.retry.maxRetryCount`|The maximum number of retries to communicate with overlord.|60|
##### SegmentWriteOutMediumFactory
When new segments are created, Druid temporarily stores some pre-processed data in some buffers. Currently two types of
*medium* exist for those buffers: *temporary files* and *off-heap memory*.
*Temporary files* (`tmpFile`) are stored under the task working directory (see `druid.indexer.task.baseTaskDir`
configuration above) and thus share it's mounting properies, e. g. they could be backed by HDD, SSD or memory (tmpfs).
This type of medium may do unnecessary disk I/O and requires some disk space to be available.
*Off-heap memory medium* (`offHeapMemory`) creates buffers in off-heap memory of a JVM process that is running a task.
This type of medium is preferred, but it may require to allow the JVM to have more off-heap memory, by changing
`-XX:MaxDirectMemorySize` configuration. It is not yet understood how does the required off-heap memory size relates
to the size of the segments being created. But definitely it doesn't make sense to add more extra off-heap memory,
than the configured maximum *heap* size (`-Xmx`) for the same JVM.
For most types of tasks SegmentWriteOutMediumFactory could be configured per-task (see [Tasks](../ingestion/tasks.html)
page, "TuningConfig" section), but if it's not specified for a task, or it's not supported for a particular task type,
then the value from the configuration below is used:
|Property|Description|Default|
|--------|-----------|-------|
|`druid.peon.defaultSegmentWriteOutMediumFactory`|`tmpFile` or `offHeapMemory`, see explanation above|`tmpFile`|

View File

@ -129,6 +129,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|`httpTimeout`|ISO8601 Period|How long to wait for a HTTP response from an indexing task.|no (default == PT10S)|
|`shutdownTimeout`|ISO8601 Period|How long to wait for the supervisor to attempt a graceful shutdown of tasks before exiting.|no (default == PT80S)|
|`offsetFetchPeriod`|ISO8601 Period|How often the supervisor queries Kafka and the indexing tasks to fetch current offsets and calculate lag.|no (default == PT30S, min == PT5S)|
|`segmentWriteOutMediumFactory`|String|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used)|
#### IndexSpec

View File

@ -154,6 +154,7 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|reportParseExceptions|Boolean|If true, exceptions encountered during parsing will be thrown and will halt ingestion. If false, unparseable rows and fields will be skipped. If an entire row is skipped, the "unparseable" counter will be incremented. If some fields in a row were parseable and some were not, the parseable fields will be indexed and the "unparseable" counter will not be incremented.|no (default == false)|
|handoffConditionTimeout|long|Milliseconds to wait for segment handoff. It must be >= 0, where 0 means to wait forever.|no (default == 0)|
|alertTimeout|long|Milliseconds timeout after which an alert is created if the task isn't finished by then. This allows users to monitor tasks that are failing to finish and give up the worker slot for any unexpected errors.|no (default == 0)|
|segmentWriteOutMediumFactory|String|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|no (not specified by default, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used)|
|indexSpec|Object|Tune how data is indexed. See below for more information.|no|
Before enabling thread priority settings, users are highly encouraged to read the [original pull request](https://github.com/druid-io/druid/pull/984) and other documentation about proper use of `-XX:+UseThreadPriorities`.

View File

@ -143,9 +143,10 @@ The tuningConfig is optional and default parameters will be used if no tuningCon
|indexSpec|defines segment storage format options to be used at indexing time, see [IndexSpec](#indexspec)|null|no|
|maxPendingPersists|Maximum number of persists that can be pending but not started. If this limit would be exceeded by a new intermediate persist, ingestion will block until the currently-running persist finishes. Maximum heap memory usage for indexing scales with maxRowsInMemory * (2 + maxPendingPersists).|0 (meaning one persist can be running concurrently with ingestion, and none can be queued up)|no|
|forceExtendableShardSpecs|Forces use of extendable shardSpecs. Experimental feature intended for use with the [Kafka indexing service extension](../development/extensions-core/kafka-ingestion.html).|false|no|
|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](./design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no|
|forceGuaranteedRollup|Forces guaranteeing the [perfect rollup](../design/index.html). The perfect rollup optimizes the total size of generated segments and querying time while indexing time will be increased. This flag cannot be used with either `appendToExisting` of IOConfig or `forceExtendableShardSpecs`. For more details, see the below __Segment publishing modes__ section.|false|no|
|reportParseExceptions|If true, exceptions encountered during parsing will be thrown and will halt ingestion; if false, unparseable rows and fields will be skipped.|false|no|
|publishTimeout|Milliseconds to wait for publishing segments. It must be >= 0, where 0 means to wait forever.|0|no|
|segmentWriteOutMediumFactory|Segment write-out medium to use when creating segments. See [Indexing Service Configuration](../configuration/indexing-service.html) page, "SegmentWriteOutMediumFactory" section for explanation and available options.|Not specified, the value from `druid.peon.defaultSegmentWriteOutMediumFactory` is used|no|
#### IndexSpec

View File

@ -21,10 +21,10 @@ package io.druid.query.aggregation.datasketches.theta;
import com.yahoo.sketches.theta.Sketch;
import io.druid.data.input.InputRow;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.GenericColumnSerializer;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
@ -80,9 +80,9 @@ public class SketchMergeComplexMetricSerde extends ComplexMetricSerde
}
@Override
public GenericColumnSerializer getSerializer(IOPeon peon, String column)
public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy());
return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy());
}
}

View File

@ -21,10 +21,10 @@ package io.druid.query.aggregation.histogram;
import com.google.common.collect.Ordering;
import io.druid.data.input.InputRow;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.GenericColumnSerializer;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
@ -102,9 +102,9 @@ public class ApproximateHistogramFoldingSerde extends ComplexMetricSerde
}
@Override
public GenericColumnSerializer getSerializer(IOPeon peon, String column)
public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy());
return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy());
}
@Override

View File

@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.java.util.common.guava.Sequences;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.query.Druids;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
@ -39,7 +40,6 @@ import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.spec.MultipleIntervalSegmentSpec;
import io.druid.segment.IndexBuilder;
import io.druid.segment.QueryableIndex;
import io.druid.segment.TestHelper;
import io.druid.segment.column.ValueType;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.virtual.ExpressionVirtualColumn;
@ -92,7 +92,7 @@ public class QuantileSqlAggregatorTest
final QueryableIndex index = IndexBuilder.create()
.tmpDir(temporaryFolder.newFolder())
.indexMerger(TestHelper.getTestIndexMergerV9())
.segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance())
.schema(
new IncrementalIndexSchema.Builder()
.withMetrics(

View File

@ -25,9 +25,12 @@ import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.RealtimeTuningConfig;
import io.druid.segment.indexing.TuningConfig;
import io.druid.segment.realtime.appenderator.AppenderatorConfig;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.io.File;
import java.util.Objects;
public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
{
@ -44,20 +47,23 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
private final boolean reportParseExceptions;
private final long handoffConditionTimeout;
private final boolean resetOffsetAutomatically;
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
@JsonCreator
public KafkaTuningConfig(
@JsonProperty("maxRowsInMemory") Integer maxRowsInMemory,
@JsonProperty("maxRowsPerSegment") Integer maxRowsPerSegment,
@JsonProperty("intermediatePersistPeriod") Period intermediatePersistPeriod,
@JsonProperty("basePersistDirectory") File basePersistDirectory,
@JsonProperty("maxPendingPersists") Integer maxPendingPersists,
@JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory,
@JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment,
@JsonProperty("intermediatePersistPeriod") @Nullable Period intermediatePersistPeriod,
@JsonProperty("basePersistDirectory") @Nullable File basePersistDirectory,
@JsonProperty("maxPendingPersists") @Nullable Integer maxPendingPersists,
@JsonProperty("indexSpec") @Nullable IndexSpec indexSpec,
// This parameter is left for compatibility when reading existing configs, to be removed in Druid 0.12.
@JsonProperty("buildV9Directly") Boolean buildV9Directly,
@JsonProperty("reportParseExceptions") Boolean reportParseExceptions,
@JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout,
@JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically
@JsonProperty("buildV9Directly") @Nullable Boolean buildV9Directly,
@JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions,
@JsonProperty("handoffConditionTimeout") @Nullable Long handoffConditionTimeout,
@JsonProperty("resetOffsetAutomatically") @Nullable Boolean resetOffsetAutomatically,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
{
// Cannot be a static because default basePersistDirectory is unique per-instance
@ -80,6 +86,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
this.resetOffsetAutomatically = resetOffsetAutomatically == null
? DEFAULT_RESET_OFFSET_AUTOMATICALLY
: resetOffsetAutomatically;
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
}
public static KafkaTuningConfig copyOf(KafkaTuningConfig config)
@ -94,7 +101,8 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
true,
config.reportParseExceptions,
config.handoffConditionTimeout,
config.resetOffsetAutomatically
config.resetOffsetAutomatically,
config.segmentWriteOutMediumFactory
);
}
@ -169,6 +177,14 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
return resetOffsetAutomatically;
}
@Override
@JsonProperty
@Nullable
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
{
return segmentWriteOutMediumFactory;
}
public KafkaTuningConfig withBasePersistDirectory(File dir)
{
return new KafkaTuningConfig(
@ -181,7 +197,8 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
true,
reportParseExceptions,
handoffConditionTimeout,
resetOffsetAutomatically
resetOffsetAutomatically,
segmentWriteOutMediumFactory
);
}
@ -194,54 +211,34 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
if (o == null || getClass() != o.getClass()) {
return false;
}
KafkaTuningConfig that = (KafkaTuningConfig) o;
if (maxRowsInMemory != that.maxRowsInMemory) {
return false;
}
if (maxRowsPerSegment != that.maxRowsPerSegment) {
return false;
}
if (maxPendingPersists != that.maxPendingPersists) {
return false;
}
if (reportParseExceptions != that.reportParseExceptions) {
return false;
}
if (handoffConditionTimeout != that.handoffConditionTimeout) {
return false;
}
if (resetOffsetAutomatically != that.resetOffsetAutomatically) {
return false;
}
if (intermediatePersistPeriod != null
? !intermediatePersistPeriod.equals(that.intermediatePersistPeriod)
: that.intermediatePersistPeriod != null) {
return false;
}
if (basePersistDirectory != null
? !basePersistDirectory.equals(that.basePersistDirectory)
: that.basePersistDirectory != null) {
return false;
}
return indexSpec != null ? indexSpec.equals(that.indexSpec) : that.indexSpec == null;
return maxRowsInMemory == that.maxRowsInMemory &&
maxRowsPerSegment == that.maxRowsPerSegment &&
maxPendingPersists == that.maxPendingPersists &&
reportParseExceptions == that.reportParseExceptions &&
handoffConditionTimeout == that.handoffConditionTimeout &&
resetOffsetAutomatically == that.resetOffsetAutomatically &&
Objects.equals(intermediatePersistPeriod, that.intermediatePersistPeriod) &&
Objects.equals(basePersistDirectory, that.basePersistDirectory) &&
Objects.equals(indexSpec, that.indexSpec) &&
Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory);
}
@Override
public int hashCode()
{
int result = maxRowsInMemory;
result = 31 * result + maxRowsPerSegment;
result = 31 * result + (intermediatePersistPeriod != null ? intermediatePersistPeriod.hashCode() : 0);
result = 31 * result + (basePersistDirectory != null ? basePersistDirectory.hashCode() : 0);
result = 31 * result + maxPendingPersists;
result = 31 * result + (indexSpec != null ? indexSpec.hashCode() : 0);
result = 31 * result + (reportParseExceptions ? 1 : 0);
result = 31 * result + (int) (handoffConditionTimeout ^ (handoffConditionTimeout >>> 32));
result = 31 * result + (resetOffsetAutomatically ? 1 : 0);
return result;
return Objects.hash(
maxRowsInMemory,
maxRowsPerSegment,
intermediatePersistPeriod,
basePersistDirectory,
maxPendingPersists,
indexSpec,
reportParseExceptions,
handoffConditionTimeout,
resetOffsetAutomatically,
segmentWriteOutMediumFactory
);
}
@Override
@ -257,6 +254,7 @@ public class KafkaTuningConfig implements TuningConfig, AppenderatorConfig
", reportParseExceptions=" + reportParseExceptions +
", handoffConditionTimeout=" + handoffConditionTimeout +
", resetOffsetAutomatically=" + resetOffsetAutomatically +
", segmentWriteOutMediumFactory=" + segmentWriteOutMediumFactory +
'}';
}
}

View File

@ -88,6 +88,7 @@ public class KafkaSupervisorSpec implements SupervisorSpec
null,
null,
null,
null,
null
);
this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig");

View File

@ -21,10 +21,12 @@ package io.druid.indexing.kafka.supervisor;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.indexing.kafka.KafkaTuningConfig;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.segment.IndexSpec;
import org.joda.time.Duration;
import org.joda.time.Period;
import javax.annotation.Nullable;
import java.io.File;
public class KafkaSupervisorTuningConfig extends KafkaTuningConfig
@ -48,6 +50,7 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig
@JsonProperty("reportParseExceptions") Boolean reportParseExceptions,
@JsonProperty("handoffConditionTimeout") Long handoffConditionTimeout,
@JsonProperty("resetOffsetAutomatically") Boolean resetOffsetAutomatically,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
@JsonProperty("workerThreads") Integer workerThreads,
@JsonProperty("chatThreads") Integer chatThreads,
@JsonProperty("chatRetries") Long chatRetries,
@ -66,7 +69,8 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig
true,
reportParseExceptions,
handoffConditionTimeout,
resetOffsetAutomatically
resetOffsetAutomatically,
segmentWriteOutMediumFactory
);
this.workerThreads = workerThreads;
@ -126,6 +130,7 @@ public class KafkaSupervisorTuningConfig extends KafkaTuningConfig
", reportParseExceptions=" + isReportParseExceptions() +
", handoffConditionTimeout=" + getHandoffConditionTimeout() +
", resetOffsetAutomatically=" + isResetOffsetAutomatically() +
", segmentWriteOutMediumFactory=" + getSegmentWriteOutMediumFactory() +
", workerThreads=" + workerThreads +
", chatThreads=" + chatThreads +
", chatRetries=" + chatRetries +

View File

@ -1629,7 +1629,8 @@ public class KafkaIndexTaskTest
true,
reportParseExceptions,
handoffConditionTimeout,
resetOffsetAutomatically
resetOffsetAutomatically,
null
);
final Map<String, Object> context = isIncrementalHandoffSupported
? ImmutableMap.of(KafkaSupervisor.IS_INCREMENTAL_HANDOFF_SUPPORTED, true)

View File

@ -111,6 +111,7 @@ public class KafkaTuningConfigTest
true,
true,
5L,
null,
null
);
KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original);

View File

@ -194,6 +194,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
false,
null,
null,
null,
numThreads,
TEST_CHAT_THREADS,
TEST_CHAT_RETRIES,

View File

@ -21,10 +21,10 @@ package io.druid.query.aggregation.variance;
import com.google.common.collect.Ordering;
import io.druid.data.input.InputRow;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.GenericColumnSerializer;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
@ -122,9 +122,9 @@ public class VarianceSerde extends ComplexMetricSerde
}
@Override
public GenericColumnSerializer getSerializer(IOPeon peon, String column)
public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy());
return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy());
}
}

View File

@ -501,7 +501,7 @@ public class IndexGeneratorJob implements Jobby
) throws IOException
{
return HadoopDruidIndexerConfig.INDEX_MERGER_V9.persist(
index, interval, file, config.getIndexSpec(), progressIndicator
index, interval, file, config.getIndexSpec(), progressIndicator, null
);
}
@ -514,7 +514,7 @@ public class IndexGeneratorJob implements Jobby
{
boolean rollup = config.getSchema().getDataSchema().getGranularitySpec().isRollup();
return HadoopDruidIndexerConfig.INDEX_MERGER_V9.mergeQueryableIndex(
indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator
indexes, rollup, aggs, file, config.getIndexSpec(), progressIndicator, null
);
}

View File

@ -535,7 +535,8 @@ public class HadoopConverterJob
inDir,
outDir,
config.getIndexSpec(),
JobHelper.progressIndicatorForContext(context)
JobHelper.progressIndicatorForContext(context),
null
);
}
catch (Exception e) {

View File

@ -56,8 +56,8 @@ import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
import io.druid.segment.IndexSpec;
import io.druid.segment.TestIndex;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.RoaringBitmapSerdeFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -293,8 +293,8 @@ public class HadoopConverterJobTest
DATASOURCE,
interval,
new IndexSpec(new RoaringBitmapSerdeFactory(null),
CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED,
CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED,
CompressionStrategy.UNCOMPRESSED,
CompressionStrategy.UNCOMPRESSED,
CompressionFactory.LongEncodingStrategy.LONGS),
oldSemgments,
true,
@ -399,8 +399,8 @@ public class HadoopConverterJobTest
DATASOURCE,
interval,
new IndexSpec(new RoaringBitmapSerdeFactory(null),
CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED,
CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED,
CompressionStrategy.UNCOMPRESSED,
CompressionStrategy.UNCOMPRESSED,
CompressionFactory.LongEncodingStrategy.LONGS),
oldSemgments,
true,

View File

@ -182,7 +182,14 @@ public class YeOldePlumberSchool implements PlumberSchool
}
fileToUpload = new File(tmpSegmentDir, "merged");
indexMergerV9.mergeQueryableIndex(indexes, schema.getGranularitySpec().isRollup(), schema.getAggregators(), fileToUpload, config.getIndexSpec());
indexMergerV9.mergeQueryableIndex(
indexes,
schema.getGranularitySpec().isRollup(),
schema.getAggregators(),
fileToUpload,
config.getIndexSpec(),
config.getSegmentWriteOutMediumFactory()
);
}
// Map merged segment so we can extract dimensions
@ -230,7 +237,8 @@ public class YeOldePlumberSchool implements PlumberSchool
indexMergerV9.persist(
indexToPersist.getIndex(),
dirToPersist,
config.getIndexSpec()
config.getIndexSpec(),
config.getSegmentWriteOutMediumFactory()
);
indexToPersist.swapSegment(null);

View File

@ -29,6 +29,7 @@ import com.google.common.collect.Lists;
import io.druid.indexing.common.TaskToolbox;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.guava.Comparators;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
@ -63,10 +64,11 @@ public class AppendTask extends MergeTaskBase
@JsonProperty("indexSpec") IndexSpec indexSpec,
// This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12.
@JsonProperty("buildV9Directly") Boolean buildV9Directly,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
@JsonProperty("context") Map<String, Object> context
)
{
super(id, dataSource, segments, context);
super(id, dataSource, segments, segmentWriteOutMediumFactory, context);
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
this.aggregators = aggregators;
}
@ -139,7 +141,8 @@ public class AppendTask extends MergeTaskBase
adapters,
aggregators == null ? null : aggregators.toArray(new AggregatorFactory[aggregators.size()]),
outDir,
indexSpec
indexSpec,
getSegmentWriteOutMediumFactory()
);
}

View File

@ -21,10 +21,13 @@ package io.druid.indexing.common.task;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.segment.IndexSpec;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@Deprecated
public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask
{
@ -36,7 +39,8 @@ public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask
@JsonProperty("segment") DataSegment segment,
@JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("force") Boolean force,
@JsonProperty("validate") Boolean validate
@JsonProperty("validate") Boolean validate,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
{
super(
@ -47,6 +51,7 @@ public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask
indexSpec,
force == null ? false : force,
validate == null ? false : validate,
segmentWriteOutMediumFactory,
null
);
}
@ -60,10 +65,11 @@ public class ConvertSegmentBackwardsCompatibleTask extends ConvertSegmentTask
@JsonProperty("segment") DataSegment segment,
@JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("force") Boolean force,
@JsonProperty("validate") Boolean validate
@JsonProperty("validate") Boolean validate,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
{
super(groupId, segment, indexSpec, force, validate, null);
super(groupId, segment, indexSpec, force, validate, segmentWriteOutMediumFactory, null);
}
}
}

View File

@ -36,12 +36,14 @@ import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.FunctionalIterable;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.segment.IndexIO;
import io.druid.segment.IndexSpec;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.Collections;
@ -61,12 +63,6 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
private static final Logger log = new Logger(ConvertSegmentTask.class);
@JsonIgnore
private final DataSegment segment;
private final IndexSpec indexSpec;
private final boolean force;
private final boolean validate;
/**
* Create a segment converter task to convert a segment to the most recent version including the specified indexSpec
*
@ -84,11 +80,22 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
IndexSpec indexSpec,
boolean force,
boolean validate,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
Map<String, Object> context
)
{
final String id = makeId(dataSource, interval);
return new ConvertSegmentTask(id, dataSource, interval, null, indexSpec, force, validate, context);
return new ConvertSegmentTask(
id,
dataSource,
interval,
null,
indexSpec,
force,
validate,
segmentWriteOutMediumFactory,
context
);
}
/**
@ -106,13 +113,24 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
IndexSpec indexSpec,
boolean force,
boolean validate,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
Map<String, Object> context
)
{
final Interval interval = segment.getInterval();
final String dataSource = segment.getDataSource();
final String id = makeId(dataSource, interval);
return new ConvertSegmentTask(id, dataSource, interval, segment, indexSpec, force, validate, context);
return new ConvertSegmentTask(
id,
dataSource,
interval,
segment,
indexSpec,
force,
validate,
segmentWriteOutMediumFactory,
context
);
}
protected static String makeId(String dataSource, Interval interval)
@ -131,22 +149,41 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
@JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("force") Boolean force,
@JsonProperty("validate") Boolean validate,
@JsonProperty("context") Map<String, Object> context
@JsonProperty("context") Map<String, Object> context,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
{
final boolean isForce = force == null ? false : force;
final boolean isValidate = validate == null ? true : validate;
if (id == null) {
if (segment == null) {
return create(dataSource, interval, indexSpec, isForce, isValidate, context);
return create(dataSource, interval, indexSpec, isForce, isValidate, segmentWriteOutMediumFactory, context);
} else {
return create(segment, indexSpec, isForce, isValidate, context);
return create(segment, indexSpec, isForce, isValidate, segmentWriteOutMediumFactory, context);
}
}
return new ConvertSegmentTask(id, dataSource, interval, segment, indexSpec, isForce, isValidate, context);
return new ConvertSegmentTask(
id,
dataSource,
interval,
segment,
indexSpec,
isForce,
isValidate,
segmentWriteOutMediumFactory,
context
);
}
protected ConvertSegmentTask(
@JsonIgnore
private final DataSegment segment;
private final IndexSpec indexSpec;
private final boolean force;
private final boolean validate;
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
ConvertSegmentTask(
String id,
String dataSource,
Interval interval,
@ -154,6 +191,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
IndexSpec indexSpec,
boolean force,
boolean validate,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
Map<String, Object> context
)
{
@ -162,6 +200,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
this.force = force;
this.validate = validate;
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
}
@JsonProperty
@ -194,6 +233,13 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
return segment;
}
@JsonProperty
@Nullable
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
{
return segmentWriteOutMediumFactory;
}
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
@ -261,7 +307,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
@Override
public Task apply(DataSegment input)
{
return new SubTask(groupId, input, indexSpec, force, validate, context);
return new SubTask(groupId, input, indexSpec, force, validate, segmentWriteOutMediumFactory, context);
}
}
);
@ -293,6 +339,8 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
private final IndexSpec indexSpec;
private final boolean force;
private final boolean validate;
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
@JsonCreator
public SubTask(
@ -301,6 +349,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
@JsonProperty("indexSpec") IndexSpec indexSpec,
@JsonProperty("force") Boolean force,
@JsonProperty("validate") Boolean validate,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
@JsonProperty("context") Map<String, Object> context
)
{
@ -321,6 +370,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
this.force = force == null ? false : force;
this.validate = validate == null ? true : validate;
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
}
@JsonProperty
@ -352,7 +402,7 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
{
log.info("Subs are good! Italian BMT and Meatball are probably my favorite.");
try {
convertSegment(toolbox, segment, indexSpec, force, validate);
convertSegment(toolbox);
}
catch (Exception e) {
log.error(e, "Conversion failed.");
@ -360,48 +410,42 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
}
return success();
}
}
private static void convertSegment(
TaskToolbox toolbox,
final DataSegment segment,
IndexSpec indexSpec,
boolean force,
boolean validate
)
throws SegmentLoadingException, IOException
{
log.info("Converting segment[%s]", segment);
final TaskActionClient actionClient = toolbox.getTaskActionClient();
final List<DataSegment> 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<DataSegment> 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<DataSegment, File> localSegments = toolbox.fetchSegments(Collections.singletonList(segment));
final Map<DataSegment, File> localSegments = toolbox.fetchSegments(Collections.singletonList(segment));
final File location = localSegments.get(segment);
final File outLocation = new File(location, "v9_out");
if (toolbox.getIndexIO().convertSegment(location, outLocation, indexSpec, force, validate)) {
final int outVersion = IndexIO.getVersionFromDir(outLocation);
final File location = localSegments.get(segment);
final File outLocation = new File(location, "v9_out");
IndexIO indexIO = toolbox.getIndexIO();
if (indexIO.convertSegment(location, outLocation, indexSpec, force, validate, segmentWriteOutMediumFactory)) {
final int outVersion = IndexIO.getVersionFromDir(outLocation);
// Appending to the version makes a new version that inherits most comparability parameters of the original
// version, but is "newer" than said original version.
DataSegment updatedSegment = segment.withVersion(StringUtils.format("%s_v%s", segment.getVersion(), outVersion));
updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment);
// Appending to the version makes a new version that inherits most comparability parameters of the original
// version, but is "newer" than said original version.
DataSegment updatedSegment = segment.withVersion(StringUtils.format("%s_v%s", segment.getVersion(), outVersion));
updatedSegment = toolbox.getSegmentPusher().push(outLocation, updatedSegment);
actionClient.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment)));
} else {
log.info("Conversion failed.");
actionClient.submit(new SegmentInsertAction(Sets.newHashSet(updatedSegment)));
} else {
log.info("Conversion failed.");
}
}
}
}

View File

@ -34,10 +34,12 @@ import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.java.util.common.UOE;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.segment.IndexSpec;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
@ -52,6 +54,12 @@ public class HadoopConverterTask extends ConvertSegmentTask
private static final String TYPE = "hadoop_convert_segment";
private static final Logger log = new Logger(HadoopConverterTask.class);
private final List<String> hadoopDependencyCoordinates;
private final URI distributedSuccessCache;
private final String jobPriority;
private final String segmentOutputPath;
private final String classpathPrefix;
@JsonCreator
public HadoopConverterTask(
@JsonProperty("id") String id,
@ -65,6 +73,7 @@ public class HadoopConverterTask extends ConvertSegmentTask
@JsonProperty("jobPriority") String jobPriority,
@JsonProperty("segmentOutputPath") String segmentOutputPath,
@JsonProperty("classpathPrefix") String classpathPrefix,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
@JsonProperty("context") Map<String, Object> context
)
{
@ -81,6 +90,7 @@ public class HadoopConverterTask extends ConvertSegmentTask
indexSpec,
force,
validate == null ? true : validate,
segmentWriteOutMediumFactory,
context
);
this.hadoopDependencyCoordinates = hadoopDependencyCoordinates;
@ -90,12 +100,6 @@ public class HadoopConverterTask extends ConvertSegmentTask
this.classpathPrefix = classpathPrefix;
}
private final List<String> hadoopDependencyCoordinates;
private final URI distributedSuccessCache;
private final String jobPriority;
private final String segmentOutputPath;
private final String classpathPrefix;
@JsonProperty
public List<String> getHadoopDependencyCoordinates()
{

View File

@ -57,6 +57,7 @@ import io.druid.java.util.common.granularity.Granularity;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.logger.Logger;
import io.druid.java.util.common.parsers.ParseException;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.query.DruidMetrics;
import io.druid.segment.IndexSpec;
import io.druid.segment.indexing.DataSchema;
@ -115,6 +116,22 @@ public class IndexTask extends AbstractTask
private static final HashFunction hashFunction = Hashing.murmur3_128();
private static final String TYPE = "index";
private static String makeGroupId(IndexIngestionSpec ingestionSchema)
{
return makeGroupId(ingestionSchema.ioConfig.appendToExisting, ingestionSchema.dataSchema.getDataSource());
}
private static String makeGroupId(boolean isAppendToExisting, String dataSource)
{
if (isAppendToExisting) {
// Shared locking group for all tasks that append, since they are OK to run concurrently.
return StringUtils.format("%s_append_%s", TYPE, dataSource);
} else {
// Return null, one locking group per task.
return null;
}
}
@JsonIgnore
private final IndexIngestionSpec ingestionSchema;
@ -156,22 +173,6 @@ public class IndexTask extends AbstractTask
this.ingestionSchema = ingestionSchema;
}
private static String makeGroupId(IndexIngestionSpec ingestionSchema)
{
return makeGroupId(ingestionSchema.ioConfig.appendToExisting, ingestionSchema.dataSchema.getDataSource());
}
private static String makeGroupId(boolean isAppendToExisting, String dataSource)
{
if (isAppendToExisting) {
// Shared locking group for all tasks that append, since they are OK to run concurrently.
return StringUtils.format("%s_append_%s", TYPE, dataSource);
} else {
// Return null, one locking group per task.
return null;
}
}
@Override
public int getPriority()
{
@ -964,6 +965,8 @@ public class IndexTask extends AbstractTask
private final boolean forceGuaranteedRollup;
private final boolean reportParseExceptions;
private final long publishTimeout;
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
@JsonCreator
public IndexTuningConfig(
@ -979,7 +982,8 @@ public class IndexTask extends AbstractTask
@JsonProperty("forceExtendableShardSpecs") @Nullable Boolean forceExtendableShardSpecs,
@JsonProperty("forceGuaranteedRollup") @Nullable Boolean forceGuaranteedRollup,
@JsonProperty("reportParseExceptions") @Nullable Boolean reportParseExceptions,
@JsonProperty("publishTimeout") @Nullable Long publishTimeout
@JsonProperty("publishTimeout") @Nullable Long publishTimeout,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
{
this(
@ -993,13 +997,14 @@ public class IndexTask extends AbstractTask
forceGuaranteedRollup,
reportParseExceptions,
publishTimeout,
null
null,
segmentWriteOutMediumFactory
);
}
private IndexTuningConfig()
{
this(null, null, null, null, null, null, null, null, null, null, null);
this(null, null, null, null, null, null, null, null, null, null, null, null);
}
private IndexTuningConfig(
@ -1013,7 +1018,8 @@ public class IndexTask extends AbstractTask
@Nullable Boolean forceGuaranteedRollup,
@Nullable Boolean reportParseExceptions,
@Nullable Long publishTimeout,
@Nullable File basePersistDirectory
@Nullable File basePersistDirectory,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
)
{
Preconditions.checkArgument(
@ -1047,6 +1053,8 @@ public class IndexTask extends AbstractTask
!(this.forceExtendableShardSpecs && this.forceGuaranteedRollup),
"Perfect rollup cannot be guaranteed with extendable shardSpecs"
);
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
}
public IndexTuningConfig withBasePersistDirectory(File dir)
@ -1062,7 +1070,8 @@ public class IndexTask extends AbstractTask
forceGuaranteedRollup,
reportParseExceptions,
publishTimeout,
dir
dir,
segmentWriteOutMediumFactory
);
}
@ -1152,64 +1161,36 @@ public class IndexTask extends AbstractTask
return new Period(Integer.MAX_VALUE); // intermediate persist doesn't make much sense for batch jobs
}
@Nullable
@Override
@JsonProperty
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
{
return segmentWriteOutMediumFactory;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final IndexTuningConfig that = (IndexTuningConfig) o;
if (!Objects.equals(targetPartitionSize, that.targetPartitionSize)) {
return false;
}
if (maxRowsInMemory != that.maxRowsInMemory) {
return false;
}
if (maxTotalRows != that.maxTotalRows) {
return false;
}
if (!Objects.equals(numShards, that.numShards)) {
return false;
}
if (!Objects.equals(indexSpec, that.indexSpec)) {
return false;
}
if (!Objects.equals(basePersistDirectory, that.basePersistDirectory)) {
return false;
}
if (maxPendingPersists != that.maxPendingPersists) {
return false;
}
if (forceExtendableShardSpecs != that.forceExtendableShardSpecs) {
return false;
}
if (forceGuaranteedRollup != that.forceGuaranteedRollup) {
return false;
}
if (reportParseExceptions != that.reportParseExceptions) {
return false;
}
if (publishTimeout != that.publishTimeout) {
return false;
}
return true;
IndexTuningConfig that = (IndexTuningConfig) o;
return maxRowsInMemory == that.maxRowsInMemory &&
maxTotalRows == that.maxTotalRows &&
maxPendingPersists == that.maxPendingPersists &&
forceExtendableShardSpecs == that.forceExtendableShardSpecs &&
forceGuaranteedRollup == that.forceGuaranteedRollup &&
reportParseExceptions == that.reportParseExceptions &&
publishTimeout == that.publishTimeout &&
Objects.equals(targetPartitionSize, that.targetPartitionSize) &&
Objects.equals(numShards, that.numShards) &&
Objects.equals(indexSpec, that.indexSpec) &&
Objects.equals(basePersistDirectory, that.basePersistDirectory) &&
Objects.equals(segmentWriteOutMediumFactory, that.segmentWriteOutMediumFactory);
}
@Override
@ -1226,7 +1207,8 @@ public class IndexTask extends AbstractTask
forceExtendableShardSpecs,
forceGuaranteedRollup,
reportParseExceptions,
publishTimeout
publishTimeout,
segmentWriteOutMediumFactory
);
}
}

View File

@ -28,6 +28,7 @@ import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import io.druid.indexing.common.TaskToolbox;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexMerger;
import io.druid.segment.IndexSpec;
@ -58,10 +59,11 @@ public class MergeTask extends MergeTaskBase
@JsonProperty("indexSpec") IndexSpec indexSpec,
// This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12.
@JsonProperty("buildV9Directly") Boolean buildV9Directly,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
@JsonProperty("context") Map<String, Object> context
)
{
super(id, dataSource, segments, context);
super(id, dataSource, segments, segmentWriteOutMediumFactory, context);
this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations");
this.rollup = rollup == null ? Boolean.TRUE : rollup;
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
@ -92,7 +94,8 @@ public class MergeTask extends MergeTaskBase
rollup,
aggregators.toArray(new AggregatorFactory[aggregators.size()]),
outDir,
indexSpec
indexSpec,
getSegmentWriteOutMediumFactory()
);
}

View File

@ -45,6 +45,7 @@ import io.druid.indexing.common.actions.TaskActionClient;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.StringUtils;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.segment.IndexIO;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
@ -61,15 +62,19 @@ import java.util.Set;
*/
public abstract class MergeTaskBase extends AbstractFixedIntervalTask
{
private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class);
@JsonIgnore
private final List<DataSegment> segments;
private static final EmittingLogger log = new EmittingLogger(MergeTaskBase.class);
@JsonIgnore
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
protected MergeTaskBase(
final String id,
final String dataSource,
final List<DataSegment> segments,
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
Map<String, Object> context
)
{
@ -104,6 +109,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
verifyInputSegments(segments);
this.segments = segments;
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
}
protected void verifyInputSegments(List<DataSegment> segments)
@ -254,6 +260,13 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
return segments;
}
@JsonProperty
@Nullable
public SegmentWriteOutMediumFactory getSegmentWriteOutMediumFactory()
{
return segmentWriteOutMediumFactory;
}
@Override
public String toString()
{
@ -262,6 +275,7 @@ public abstract class MergeTaskBase extends AbstractFixedIntervalTask
.add("dataSource", getDataSource())
.add("interval", getInterval())
.add("segments", segments)
.add("segmentWriteOutMediumFactory", segmentWriteOutMediumFactory)
.toString();
}

View File

@ -26,11 +26,13 @@ import io.druid.indexing.common.TaskStatus;
import io.druid.indexing.common.TaskToolbox;
import io.druid.indexing.common.actions.SegmentListUsedAction;
import io.druid.java.util.common.DateTimes;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.IndexSpec;
import io.druid.timeline.DataSegment;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
@ -43,6 +45,8 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask
private final List<AggregatorFactory> aggregators;
private final Boolean rollup;
private final IndexSpec indexSpec;
@Nullable
private final SegmentWriteOutMediumFactory segmentWriteOutMediumFactory;
public SameIntervalMergeTask(
@JsonProperty("id") String id,
@ -54,6 +58,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask
// This parameter is left for compatibility when reading existing JSONs, to be removed in Druid 0.12.
@SuppressWarnings("unused")
@JsonProperty("buildV9Directly") Boolean buildV9Directly,
@JsonProperty("segmentWriteOutMediumFactory") @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
@JsonProperty("context") Map<String, Object> context
)
{
@ -66,6 +71,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask
this.aggregators = Preconditions.checkNotNull(aggregators, "null aggregations");
this.rollup = rollup == null ? Boolean.TRUE : rollup;
this.indexSpec = indexSpec == null ? new IndexSpec() : indexSpec;
this.segmentWriteOutMediumFactory = segmentWriteOutMediumFactory;
}
@JsonProperty("aggregations")
@ -130,6 +136,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask
aggregators,
rollup,
indexSpec,
segmentWriteOutMediumFactory,
getContext()
);
final TaskStatus status = mergeTask.run(toolbox);
@ -141,13 +148,14 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask
public static class SubTask extends MergeTask
{
public SubTask(
private SubTask(
String baseId,
String dataSource,
List<DataSegment> segments,
List<AggregatorFactory> aggregators,
Boolean rollup,
IndexSpec indexSpec,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory,
Map<String, Object> context
)
{
@ -159,6 +167,7 @@ public class SameIntervalMergeTask extends AbstractFixedIntervalTask
rollup,
indexSpec,
true,
segmentWriteOutMediumFactory,
context
);
}

View File

@ -35,6 +35,7 @@ import io.druid.segment.IndexMergerV9;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.realtime.firehose.ChatHandlerProvider;
import io.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.server.security.AuthConfig;
import io.druid.server.security.AuthorizerMapper;
@ -56,6 +57,7 @@ public class TestUtils
this.jsonMapper = new DefaultObjectMapper();
indexIO = new IndexIO(
jsonMapper,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
new ColumnConfig()
{
@Override
@ -65,7 +67,7 @@ public class TestUtils
}
}
);
indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO);
indexMergerV9 = new IndexMergerV9(jsonMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance());
final List<? extends Module> list = new ServerModule().getJacksonModules();
for (Module module : list) {

View File

@ -69,7 +69,7 @@ import io.druid.segment.SimpleQueryableIndex;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy.CompressionStrategy;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.CompressionFactory.LongEncodingStrategy;
import io.druid.segment.data.ListIndexed;
import io.druid.segment.data.RoaringBitmapSerdeFactory;
@ -78,6 +78,7 @@ import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import io.druid.segment.loading.SegmentLoadingException;
import io.druid.segment.transform.TransformingInputRowParser;
import io.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NumberedShardSpec;
import org.hamcrest.CoreMatchers;
@ -240,7 +241,8 @@ public class CompactionTaskTest
false,
true,
false,
100L
100L,
null
);
}
@ -499,7 +501,7 @@ public class CompactionTaskTest
indexIO,
null,
null,
new IndexMergerV9(objectMapper, indexIO),
new IndexMergerV9(objectMapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()),
null,
null,
null,
@ -549,7 +551,7 @@ public class CompactionTaskTest
Map<DataSegment, File> segmentFileMap
)
{
super(mapper, () -> 0);
super(mapper, OffHeapMemorySegmentWriteOutMediumFactory.instance(), () -> 0);
queryableIndexMap = new HashMap<>(segmentFileMap.size());
for (Entry<DataSegment, File> entry : segmentFileMap.entrySet()) {

View File

@ -51,7 +51,7 @@ public class ConvertSegmentTaskTest
DateTime start = DateTimes.nowUtc();
final Interval interval = new Interval(start.minus(1000), start);
ConvertSegmentTask task = ConvertSegmentTask.create(dataSource, interval, null, false, true, null);
ConvertSegmentTask task = ConvertSegmentTask.create(dataSource, interval, null, false, true, null, null);
Task task2 = jsonMapper.readValue(jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task), Task.class);
Assert.assertEquals(task, task2);
@ -68,7 +68,7 @@ public class ConvertSegmentTaskTest
102937
);
task = ConvertSegmentTask.create(segment, null, false, true, null);
task = ConvertSegmentTask.create(segment, null, false, true, null, null);
task2 = jsonMapper.readValue(jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(task), Task.class);
Assert.assertEquals(task, task2);

View File

@ -25,8 +25,8 @@ import com.google.common.collect.ImmutableMap;
import io.druid.indexing.common.TestUtils;
import io.druid.java.util.common.Intervals;
import io.druid.segment.IndexSpec;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
import io.druid.timeline.DataSegment;
import io.druid.timeline.partition.NoneShardSpec;
@ -54,8 +54,8 @@ public class HadoopConverterTaskSerDeTest
private static final int BINARY_VERSION = 34718;
private static final long SEGMENT_SIZE = 7483901348790L;
private static final IndexSpec INDEX_SPEC = new IndexSpec(new ConciseBitmapSerdeFactory(),
CompressedObjectStrategy.CompressionStrategy.LZ4,
CompressedObjectStrategy.CompressionStrategy.LZF,
CompressionStrategy.LZ4,
CompressionStrategy.LZF,
CompressionFactory.LongEncodingStrategy.LONGS);
private static final DataSegment DATA_SEGMENT = new DataSegment(
DATA_SOURCE,
@ -97,6 +97,7 @@ public class HadoopConverterTaskSerDeTest
PRIORITY,
OUTPUT_PATH,
CLASSPATH_PREFIX,
null,
null
);
final String strOrig = jsonMapper.writeValueAsString(orig);
@ -122,6 +123,7 @@ public class HadoopConverterTaskSerDeTest
PRIORITY,
OUTPUT_PATH,
CLASSPATH_PREFIX,
null,
null
);
HadoopConverterTask.ConverterSubTask subTask = new HadoopConverterTask.ConverterSubTask(
@ -175,6 +177,7 @@ public class HadoopConverterTaskSerDeTest
PRIORITY,
OUTPUT_PATH,
CLASSPATH_PREFIX,
null,
null
);
HadoopConverterTask.ConverterSubTask subTask = new HadoopConverterTask.ConverterSubTask(
@ -203,6 +206,7 @@ public class HadoopConverterTaskSerDeTest
PRIORITY,
OUTPUT_PATH,
CLASSPATH_PREFIX,
null,
null
);
Assert.assertTrue(orig.isValidate());
@ -223,6 +227,7 @@ public class HadoopConverterTaskSerDeTest
null,
OUTPUT_PATH,
null,
null,
null
);
Assert.assertEquals(DATA_SOURCE, parent.getDataSource());
@ -248,6 +253,7 @@ public class HadoopConverterTaskSerDeTest
PRIORITY,
OUTPUT_PATH,
CLASSPATH_PREFIX,
null,
null
);
orig.getSegment();
@ -268,6 +274,7 @@ public class HadoopConverterTaskSerDeTest
null,
OUTPUT_PATH,
null,
null,
null
);
}
@ -287,6 +294,7 @@ public class HadoopConverterTaskSerDeTest
null,
OUTPUT_PATH,
null,
null,
null
);
}
@ -306,6 +314,7 @@ public class HadoopConverterTaskSerDeTest
null,
OUTPUT_PATH,
null,
null,
null
);
}
@ -325,6 +334,7 @@ public class HadoopConverterTaskSerDeTest
null,
null,
null,
null,
null
);
}

View File

@ -1147,6 +1147,7 @@ public class IndexTaskTest
forceExtendableShardSpecs,
forceGuaranteedRollup,
reportParseException,
null,
null
);
}

View File

@ -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<DataSegment, File> segments, File outDir) throws Exception

View File

@ -931,6 +931,7 @@ public class RealtimeIndexTaskTest
0,
reportParseExceptions,
handoffTimeout,
null,
null
);
return new RealtimeIndexTask(

View File

@ -91,6 +91,7 @@ public class SameIntervalMergeTaskTest
true,
indexSpec,
true,
null,
null
);

View File

@ -35,12 +35,13 @@ import io.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
import io.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.granularity.Granularities;
import io.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.CountAggregatorFactory;
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
import io.druid.segment.IndexSpec;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.RoaringBitmapSerdeFactory;
import io.druid.segment.indexing.DataSchema;
import io.druid.segment.indexing.RealtimeIOConfig;
@ -189,7 +190,7 @@ public class TaskSerdeTest
jsonMapper
),
new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true),
new IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null)
new IndexTuningConfig(10000, 10, null, 9999, null, indexSpec, 3, true, true, false, null, null, null)
),
null
);
@ -252,7 +253,7 @@ public class TaskSerdeTest
jsonMapper
),
new IndexIOConfig(new LocalFirehoseFactory(new File("lol"), "rofl", null), true),
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null)
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null)
),
null
);
@ -298,6 +299,7 @@ public class TaskSerdeTest
true,
indexSpec,
true,
null,
null
);
@ -347,6 +349,7 @@ public class TaskSerdeTest
true,
indexSpec,
true,
null,
null
);
@ -414,6 +417,7 @@ public class TaskSerdeTest
null,
false,
true,
TmpFileSegmentWriteOutMediumFactory.instance(),
null
);
@ -429,7 +433,8 @@ public class TaskSerdeTest
Assert.assertEquals(task.getGroupId(), task2.getGroupId());
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getInterval(), task2.getInterval());
Assert.assertEquals(task.getSegment(), task.getSegment());
Assert.assertEquals(task.getSegment(), task2.getSegment());
Assert.assertEquals(task.getSegmentWriteOutMediumFactory(), task2.getSegmentWriteOutMediumFactory());
}
@Test
@ -441,6 +446,7 @@ public class TaskSerdeTest
indexSpec,
false,
true,
null,
null
);
@ -503,6 +509,7 @@ public class TaskSerdeTest
0,
true,
null,
null,
null
)
),
@ -567,6 +574,7 @@ public class TaskSerdeTest
),
indexSpec,
true,
null,
null
);
@ -665,6 +673,7 @@ public class TaskSerdeTest
indexSpec,
false,
true,
TmpFileSegmentWriteOutMediumFactory.instance(),
null
);
final String json = jsonMapper.writeValueAsString(task);
@ -686,41 +695,43 @@ public class TaskSerdeTest
0,
12345L
);
final ConvertSegmentTask convertSegmentTaskOriginal = ConvertSegmentTask.create(
final ConvertSegmentTask originalTask = ConvertSegmentTask.create(
segment,
new IndexSpec(
new RoaringBitmapSerdeFactory(null),
CompressedObjectStrategy.CompressionStrategy.LZF,
CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED,
CompressionStrategy.LZF,
CompressionStrategy.UNCOMPRESSED,
CompressionFactory.LongEncodingStrategy.LONGS
),
false,
true,
TmpFileSegmentWriteOutMediumFactory.instance(),
null
);
final String json = jsonMapper.writeValueAsString(convertSegmentTaskOriginal);
final String json = jsonMapper.writeValueAsString(originalTask);
final Task task = jsonMapper.readValue(json, Task.class);
Assert.assertTrue(task instanceof ConvertSegmentTask);
final ConvertSegmentTask convertSegmentTask = (ConvertSegmentTask) task;
Assert.assertEquals(convertSegmentTaskOriginal.getDataSource(), convertSegmentTask.getDataSource());
Assert.assertEquals(convertSegmentTaskOriginal.getInterval(), convertSegmentTask.getInterval());
Assert.assertEquals(originalTask.getDataSource(), convertSegmentTask.getDataSource());
Assert.assertEquals(originalTask.getInterval(), convertSegmentTask.getInterval());
Assert.assertEquals(
convertSegmentTaskOriginal.getIndexSpec().getBitmapSerdeFactory().getClass().getCanonicalName(),
originalTask.getIndexSpec().getBitmapSerdeFactory().getClass().getCanonicalName(),
convertSegmentTask.getIndexSpec()
.getBitmapSerdeFactory()
.getClass()
.getCanonicalName()
);
Assert.assertEquals(
convertSegmentTaskOriginal.getIndexSpec().getDimensionCompression(),
originalTask.getIndexSpec().getDimensionCompression(),
convertSegmentTask.getIndexSpec().getDimensionCompression()
);
Assert.assertEquals(
convertSegmentTaskOriginal.getIndexSpec().getMetricCompression(),
originalTask.getIndexSpec().getMetricCompression(),
convertSegmentTask.getIndexSpec().getMetricCompression()
);
Assert.assertEquals(false, convertSegmentTask.isForce());
Assert.assertEquals(segment, convertSegmentTask.getSegment());
Assert.assertEquals(originalTask.getSegmentWriteOutMediumFactory(), convertSegmentTask.getSegmentWriteOutMediumFactory());
}
@Test

View File

@ -170,7 +170,7 @@ public class IngestSegmentFirehoseFactoryTest
if (!persistDir.mkdirs() && !persistDir.exists()) {
throw new IOE("Could not create directory at [%s]", persistDir.getAbsolutePath());
}
INDEX_MERGER_V9.persist(index, persistDir, indexSpec);
INDEX_MERGER_V9.persist(index, persistDir, indexSpec, null);
final IndexerSQLMetadataStorageCoordinator mdc = new IndexerSQLMetadataStorageCoordinator(null, null, null)
{

View File

@ -231,7 +231,7 @@ public class IngestSegmentFirehoseFactoryTimelineTest
}
try {
INDEX_MERGER_V9.persist(index, persistDir, new IndexSpec());
INDEX_MERGER_V9.persist(index, persistDir, new IndexSpec(), null);
}
catch (IOException e) {
throw Throwables.propagate(e);

View File

@ -670,7 +670,7 @@ public class TaskLifecycleTest
mapper
),
new IndexIOConfig(new MockFirehoseFactory(false), false),
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null)
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null)
),
null
);
@ -728,7 +728,7 @@ public class TaskLifecycleTest
mapper
),
new IndexIOConfig(new MockExceptionalFirehoseFactory(), false),
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null)
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, 3, true, true, false, null, null, null)
),
null
);
@ -1093,7 +1093,7 @@ public class TaskLifecycleTest
mapper
),
new IndexIOConfig(new MockFirehoseFactory(false), false),
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null)
new IndexTuningConfig(10000, 10, null, null, null, indexSpec, null, false, null, null, null, null, null)
),
null
);
@ -1215,6 +1215,7 @@ public class TaskLifecycleTest
0,
null,
null,
null,
null
);
FireDepartment fireDepartment = new FireDepartment(dataSchema, realtimeIOConfig, realtimeTuningConfig);

View File

@ -46,7 +46,9 @@ import java.io.OutputStreamWriter;
import java.io.Writer;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.FileChannel;
import java.nio.channels.GatheringByteChannel;
import java.nio.file.StandardOpenOption;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
@ -291,21 +293,20 @@ public class FileSmoosher implements Closeable
return new SmooshedWriter()
{
private final FileOutputStream out = new FileOutputStream(tmpFile);
private final GatheringByteChannel channel = out.getChannel();
private final Closer closer = Closer.create();
private final GatheringByteChannel channel =
FileChannel.open(
tmpFile.toPath(),
StandardOpenOption.WRITE,
StandardOpenOption.CREATE,
StandardOpenOption.TRUNCATE_EXISTING
);
private int currOffset = 0;
{
closer.register(out);
closer.register(channel);
}
@Override
public void close() throws IOException
{
closer.close();
channel.close();
completedFiles.add(tmpFile);
filesInProcess.remove(tmpFile);

View File

@ -227,7 +227,7 @@
<dependency>
<groupId>com.ning</groupId>
<artifactId>compress-lzf</artifactId>
<version>1.0.3</version>
<version>1.0.4</version>
</dependency>
<dependency>
<groupId>io.airlift</groupId>
@ -709,6 +709,8 @@
<artifactId>asm-commons</artifactId>
<version>5.2</version>
</dependency>
<!-- Test Scope -->
<dependency>
<groupId>org.codehaus.jackson</groupId>
<artifactId>jackson-core-asl</artifactId>

View File

@ -35,6 +35,11 @@
<artifactId>druid-common</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>java-util</artifactId>
<version>${project.parent.version}</version>
</dependency>
<dependency>
<groupId>io.druid</groupId>
<artifactId>druid-hll</artifactId>
@ -93,6 +98,10 @@
<groupId>org.ow2.asm</groupId>
<artifactId>asm-commons</artifactId>
</dependency>
<dependency>
<groupId>it.unimi.dsi</groupId>
<artifactId>fastutil</artifactId>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -23,10 +23,10 @@ import com.google.common.collect.Ordering;
import io.druid.data.input.InputRow;
import io.druid.hll.HyperLogLogCollector;
import io.druid.hll.HyperLogLogHash;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.GenericColumnSerializer;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.ObjectStrategy;
import io.druid.segment.serde.ComplexColumnPartSupplier;
import io.druid.segment.serde.ComplexMetricExtractor;
@ -148,9 +148,9 @@ public class HyperUniquesSerde extends ComplexMetricSerde
}
@Override
public GenericColumnSerializer getSerializer(IOPeon peon, String column)
public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column)
{
return LargeColumnSupportedComplexColumnSerializer.create(peon, column, this.getObjectStrategy());
return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy());
}
}

View File

@ -19,23 +19,26 @@
package io.druid.segment;
import com.google.common.annotations.VisibleForTesting;
import io.druid.io.Channels;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import io.druid.java.util.common.io.Closer;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressedVSizeIntsIndexedSupplier;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.IndexedIterable;
import io.druid.segment.data.IndexedMultivalue;
import io.druid.segment.data.WritableSupplier;
import it.unimi.dsi.fastutil.ints.IntArrayList;
import it.unimi.dsi.fastutil.ints.IntList;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.WritableByteChannel;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
/**
* Format -
@ -55,7 +58,7 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier<IndexedM
//values - indexed integers representing actual values in each row
private final CompressedVSizeIntsIndexedSupplier valueSupplier;
CompressedVSizeIndexedSupplier(
private CompressedVSizeIndexedSupplier(
CompressedVSizeIntsIndexedSupplier offsetSupplier,
CompressedVSizeIntsIndexedSupplier valueSupplier
)
@ -65,49 +68,49 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier<IndexedM
}
@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);
Channels.writeFully(channel, ByteBuffer.wrap(new byte[]{version}));
offsetSupplier.writeTo(channel, smoosher);
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();
if (versionFromBuffer == version) {
CompressedVSizeIntsIndexedSupplier offsetSupplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(
buffer,
order,
fileMapper
order
);
CompressedVSizeIntsIndexedSupplier valueSupplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(
buffer,
order,
fileMapper
order
);
return new CompressedVSizeIndexedSupplier(offsetSupplier, valueSupplier);
}
throw new IAE("Unknown version[%s]", versionFromBuffer);
}
@VisibleForTesting
public static CompressedVSizeIndexedSupplier fromIterable(
Iterable<IndexedInts> objectsIterable,
int maxValue,
final Iterable<IndexedInts> objectsIterable,
final int maxValue,
final ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression
final CompressionStrategy compression,
final Closer closer
)
{
Iterator<IndexedInts> objects = objectsIterable.iterator();
List<Integer> offsetList = new ArrayList<>();
List<Integer> values = new ArrayList<>();
IntList offsetList = new IntArrayList();
IntList values = new IntArrayList();
int offset = 0;
while (objects.hasNext()) {
@ -125,14 +128,16 @@ public class CompressedVSizeIndexedSupplier implements WritableSupplier<IndexedM
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);
}

View File

@ -19,22 +19,23 @@
package io.druid.segment;
import com.google.common.annotations.VisibleForTesting;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import io.druid.java.util.common.io.Closer;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.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 +51,7 @@ public class CompressedVSizeIndexedV3Supplier implements WritableSupplier<Indexe
private final CompressedIntsIndexedSupplier offsetSupplier;
private final CompressedVSizeIntsIndexedSupplier valueSupplier;
CompressedVSizeIndexedV3Supplier(
private CompressedVSizeIndexedV3Supplier(
CompressedIntsIndexedSupplier offsetSupplier,
CompressedVSizeIntsIndexedSupplier valueSupplier
)
@ -59,42 +60,37 @@ public class CompressedVSizeIndexedV3Supplier implements WritableSupplier<Indexe
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();
if (versionFromBuffer == VERSION) {
CompressedIntsIndexedSupplier offsetSupplier = CompressedIntsIndexedSupplier.fromByteBuffer(
buffer,
order,
fileMapper
order
);
CompressedVSizeIntsIndexedSupplier valueSupplier = CompressedVSizeIntsIndexedSupplier.fromByteBuffer(
buffer,
order,
fileMapper
order
);
return new CompressedVSizeIndexedV3Supplier(offsetSupplier, valueSupplier);
}
throw new IAE("Unknown version[%s]", versionFromBuffer);
}
// for test
@VisibleForTesting
public static CompressedVSizeIndexedV3Supplier fromIterable(
Iterable<IndexedInts> objectsIterable,
int offsetChunkFactor,
int maxValue,
final Iterable<IndexedInts> objectsIterable,
final int offsetChunkFactor,
final int maxValue,
final ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression
final CompressionStrategy compression,
final Closer closer
)
{
Iterator<IndexedInts> objects = objectsIterable.iterator();
List<Integer> offsetList = new ArrayList<>();
List<Integer> values = new ArrayList<>();
IntArrayList offsetList = new IntArrayList();
IntArrayList values = new IntArrayList();
int offset = 0;
while (objects.hasNext()) {
@ -110,30 +106,32 @@ public class CompressedVSizeIndexedV3Supplier implements WritableSupplier<Indexe
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

View File

@ -22,11 +22,10 @@ package io.druid.segment;
import io.druid.data.input.impl.DimensionSchema.MultiValueHandling;
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 io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
/**
@ -95,8 +94,7 @@ 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 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
@ -104,8 +102,7 @@ public interface DimensionHandler
*/
DimensionMergerV9<EncodedKeyComponentType> makeMerger(
IndexSpec indexSpec,
File outDir,
IOPeon ioPeon,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
ProgressIndicator progress
) throws IOException;

View File

@ -28,9 +28,6 @@ package io.druid.segment;
*/
public interface DoubleColumnSelector extends ColumnValueSelector<Double>
{
@Override
double getDouble();
/**
* @deprecated This method is marked as deprecated in DoubleColumnSelector to minimize the probability of accidential
* calling. "Polymorphism" of DoubleColumnSelector should be used only when operating on {@link ColumnValueSelector}

View File

@ -19,13 +19,12 @@
package io.druid.segment;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.DoubleSupplierSerializer;
import io.druid.segment.data.IOPeon;
import java.io.IOException;
import java.nio.ByteOrder;
@ -34,28 +33,28 @@ import java.nio.channels.WritableByteChannel;
public class DoubleColumnSerializer implements GenericColumnSerializer
{
public static DoubleColumnSerializer create(
IOPeon ioPeon,
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
CompressedObjectStrategy.CompressionStrategy compression
CompressionStrategy compression
)
{
return new DoubleColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression);
return new DoubleColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression);
}
private final IOPeon ioPeon;
private final SegmentWriteOutMedium segmentWriteOutMedium;
private final String filenameBase;
private final ByteOrder byteOrder;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final CompressionStrategy compression;
private DoubleSupplierSerializer writer;
public DoubleColumnSerializer(
IOPeon ioPeon,
private DoubleColumnSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression
CompressionStrategy compression
)
{
this.ioPeon = ioPeon;
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.filenameBase = filenameBase;
this.byteOrder = byteOrder;
this.compression = compression;
@ -65,7 +64,7 @@ public class DoubleColumnSerializer implements GenericColumnSerializer
public void open() throws IOException
{
writer = CompressionFactory.getDoubleSerializer(
ioPeon,
segmentWriteOutMedium,
StringUtils.format("%s.double_column", filenameBase),
byteOrder,
compression
@ -81,21 +80,15 @@ public class DoubleColumnSerializer implements GenericColumnSerializer
}
@Override
public void close() throws IOException
{
writer.close();
}
@Override
public long getSerializedSize()
public long getSerializedSize() throws IOException
{
return writer.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
writer.writeToChannel(channel, smoosher);
writer.writeTo(channel, smoosher);
}
}

View File

@ -22,11 +22,10 @@ package io.druid.segment;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.GenericColumn;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.Indexed;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
public class DoubleDimensionHandler implements DimensionHandler<Double, Double, Double>
@ -52,13 +51,16 @@ public class DoubleDimensionHandler implements DimensionHandler<Double, Double,
@Override
public DimensionMergerV9<Double> makeMerger(
IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
ProgressIndicator progress
) throws IOException
{
return new DoubleDimensionMergerV9(
dimensionName,
indexSpec,
ioPeon
segmentWriteOutMedium
);
}

View File

@ -21,9 +21,9 @@ package io.druid.segment;
import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.serde.DoubleGenericColumnPartSerde;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
import java.nio.IntBuffer;
@ -33,49 +33,32 @@ public class DoubleDimensionMergerV9 implements DimensionMergerV9<Double>
{
protected String dimensionName;
protected final IndexSpec indexSpec;
protected IOPeon ioPeon;
private DoubleColumnSerializer serializer;
public DoubleDimensionMergerV9(
String dimensionName,
IndexSpec indexSpec,
IOPeon ioPeon
SegmentWriteOutMedium segmentWriteOutMedium
)
{
this.dimensionName = dimensionName;
this.indexSpec = indexSpec;
this.ioPeon = ioPeon;
try {
setupEncodedValueWriter();
setupEncodedValueWriter(segmentWriteOutMedium);
}
catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
protected void setupEncodedValueWriter() throws IOException
private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException
{
final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression();
this.serializer = DoubleColumnSerializer.create(ioPeon, dimensionName, metCompression);
final CompressionStrategy metCompression = indexSpec.getMetricCompression();
this.serializer = DoubleColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression);
serializer.open();
}
@Override
public ColumnDescriptor makeColumnDescriptor() throws IOException
{
serializer.close();
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
builder.setValueType(ValueType.DOUBLE);
builder.addSerde(
DoubleGenericColumnPartSerde.serializerBuilder()
.withByteOrder(IndexIO.BYTE_ORDER)
.withDelegate(serializer)
.build()
);
return builder.build();
}
@Override
public void writeMergedValueMetadata(List<IndexableAdapter> adapters) throws IOException
{
@ -106,4 +89,18 @@ public class DoubleDimensionMergerV9 implements DimensionMergerV9<Double>
// 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();
}
}

View File

@ -28,9 +28,6 @@ package io.druid.segment;
*/
public interface FloatColumnSelector extends ColumnValueSelector<Float>
{
@Override
float getFloat();
/**
* @deprecated This method is marked as deprecated in FloatColumnSelector to minimize the probability of accidential
* calling. "Polymorphism" of FloatColumnSelector should be used only when operating on {@link ColumnValueSelector}

View File

@ -21,10 +21,10 @@ package io.druid.segment;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.FloatSupplierSerializer;
import io.druid.segment.data.IOPeon;
import java.io.IOException;
import java.nio.ByteOrder;
@ -33,28 +33,28 @@ import java.nio.channels.WritableByteChannel;
public class FloatColumnSerializer implements GenericColumnSerializer
{
public static FloatColumnSerializer create(
IOPeon ioPeon,
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
CompressedObjectStrategy.CompressionStrategy compression
CompressionStrategy compression
)
{
return new FloatColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression);
return new FloatColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression);
}
private final IOPeon ioPeon;
private final SegmentWriteOutMedium segmentWriteOutMedium;
private final String filenameBase;
private final ByteOrder byteOrder;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final CompressionStrategy compression;
private FloatSupplierSerializer writer;
public FloatColumnSerializer(
IOPeon ioPeon,
private FloatColumnSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression
CompressionStrategy compression
)
{
this.ioPeon = ioPeon;
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.filenameBase = filenameBase;
this.byteOrder = byteOrder;
this.compression = compression;
@ -64,7 +64,7 @@ public class FloatColumnSerializer implements GenericColumnSerializer
public void open() throws IOException
{
writer = CompressionFactory.getFloatSerializer(
ioPeon,
segmentWriteOutMedium,
StringUtils.format("%s.float_column", filenameBase),
byteOrder,
compression
@ -80,21 +80,14 @@ public class FloatColumnSerializer implements GenericColumnSerializer
}
@Override
public void close() throws IOException
{
writer.close();
}
@Override
public long getSerializedSize()
public long getSerializedSize() throws IOException
{
return writer.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
writer.writeToChannel(channel, smoosher);
writer.writeTo(channel, smoosher);
}
}

View File

@ -22,11 +22,10 @@ package io.druid.segment;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.GenericColumn;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.Indexed;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
public class FloatDimensionHandler implements DimensionHandler<Float, Float, Float>
@ -52,13 +51,16 @@ public class FloatDimensionHandler implements DimensionHandler<Float, Float, Flo
@Override
public DimensionMergerV9<Float> makeMerger(
IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
ProgressIndicator progress
) throws IOException
{
return new FloatDimensionMergerV9(
dimensionName,
indexSpec,
ioPeon
segmentWriteOutMedium
);
}

View File

@ -21,9 +21,9 @@ package io.druid.segment;
import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.serde.FloatGenericColumnPartSerde;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
import java.nio.IntBuffer;
@ -33,32 +33,29 @@ public class FloatDimensionMergerV9 implements DimensionMergerV9<Float>
{
protected String dimensionName;
protected final IndexSpec indexSpec;
protected IOPeon ioPeon;
private FloatColumnSerializer serializer;
public FloatDimensionMergerV9(
String dimensionName,
IndexSpec indexSpec,
IOPeon ioPeon
SegmentWriteOutMedium segmentWriteOutMedium
)
{
this.dimensionName = dimensionName;
this.indexSpec = indexSpec;
this.ioPeon = ioPeon;
try {
setupEncodedValueWriter();
setupEncodedValueWriter(segmentWriteOutMedium);
}
catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
protected void setupEncodedValueWriter() throws IOException
private void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException
{
final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression();
this.serializer = FloatColumnSerializer.create(ioPeon, dimensionName, metCompression);
final CompressionStrategy metCompression = indexSpec.getMetricCompression();
this.serializer = FloatColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression);
serializer.open();
}
@ -96,7 +93,6 @@ public class FloatDimensionMergerV9 implements DimensionMergerV9<Float>
@Override
public ColumnDescriptor makeColumnDescriptor() throws IOException
{
serializer.close();
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
builder.setValueType(ValueType.FLOAT);
builder.addSerde(

View File

@ -20,20 +20,14 @@
package io.druid.segment;
import io.druid.guice.annotations.ExtensionPoint;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.serde.Serializer;
import java.io.Closeable;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;
@ExtensionPoint
public interface GenericColumnSerializer extends Closeable
public interface GenericColumnSerializer extends Serializer
{
void open() throws IOException;
void serialize(Object obj) throws IOException;
long getSerializedSize();
void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException;
}

View File

@ -44,6 +44,7 @@ import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.io.smoosh.Smoosh;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import io.druid.java.util.common.logger.Logger;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnBuilder;
import io.druid.segment.column.ColumnCapabilities;
@ -70,6 +71,7 @@ import io.druid.segment.serde.LongGenericColumnSupplier;
import io.druid.segment.serde.SpatialIndexColumnPartSupplier;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
@ -95,11 +97,13 @@ public class IndexIO
private static final SerializerUtils serializerUtils = new SerializerUtils();
private final ObjectMapper mapper;
private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory;
@Inject
public IndexIO(ObjectMapper mapper, ColumnConfig columnConfig)
public IndexIO(ObjectMapper mapper, SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory, ColumnConfig columnConfig)
{
this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper");
this.defaultSegmentWriteOutMediumFactory = Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory");
Preconditions.checkNotNull(columnConfig, "null ColumnConfig");
ImmutableMap.Builder<Integer, IndexLoader> indexLoadersBuilder = ImmutableMap.builder();
LegacyIndexLoader legacyIndexLoader = new LegacyIndexLoader(new DefaultIndexIOHandler(), columnConfig);
@ -224,13 +228,17 @@ public class IndexIO
File converted,
IndexSpec indexSpec,
boolean forceIfCurrent,
boolean validate
boolean validate,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
final int version = SegmentUtils.getVersionFromDir(toConvert);
boolean current = version == CURRENT_VERSION_ID;
if (!current || forceIfCurrent) {
new IndexMergerV9(mapper, this).convert(toConvert, converted, indexSpec);
if (segmentWriteOutMediumFactory == null) {
segmentWriteOutMediumFactory = this.defaultSegmentWriteOutMediumFactory;
}
new IndexMergerV9(mapper, this, segmentWriteOutMediumFactory).convert(toConvert, converted, indexSpec);
if (validate) {
validateTwoSegments(toConvert, converted);
}
@ -341,8 +349,7 @@ public class IndexIO
CompressedLongsIndexedSupplier timestamps = CompressedLongsIndexedSupplier.fromByteBuffer(
smooshedFiles.mapFile(makeTimeFile(inDir, BYTE_ORDER).getName()),
BYTE_ORDER,
smooshedFiles
BYTE_ORDER
);
Map<String, MetricHolder> metrics = Maps.newLinkedHashMap();

View File

@ -19,6 +19,7 @@
package io.druid.segment;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
@ -39,6 +40,7 @@ import io.druid.java.util.common.parsers.CloseableIterator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.segment.data.Indexed;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
import it.unimi.dsi.fastutil.ints.IntIterator;
import it.unimi.dsi.fastutil.ints.IntSortedSet;
@ -156,7 +158,12 @@ public interface IndexMerger
return Lists.newArrayList(retVal);
}
File persist(IncrementalIndex index, File outDir, IndexSpec indexSpec) throws IOException;
File persist(
IncrementalIndex index,
File outDir,
IndexSpec indexSpec,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException;
/**
* This is *not* thread-safe and havok will ensue if this is called and writes are still occurring
@ -170,22 +177,21 @@ public interface IndexMerger
*
* @throws IOException if an IO error occurs persisting the index
*/
File persist(IncrementalIndex index, Interval dataInterval, File outDir, IndexSpec indexSpec) throws IOException;
File persist(
IncrementalIndex index,
Interval dataInterval,
File outDir,
IndexSpec indexSpec,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException;
File persist(
IncrementalIndex index,
Interval dataInterval,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress
) throws IOException;
File mergeQueryableIndex(
List<QueryableIndex> indexes,
boolean rollup,
AggregatorFactory[] metricAggs,
File outDir,
IndexSpec indexSpec
ProgressIndicator progress,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException;
File mergeQueryableIndex(
@ -194,9 +200,20 @@ public interface IndexMerger
AggregatorFactory[] metricAggs,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException;
File mergeQueryableIndex(
List<QueryableIndex> indexes,
boolean rollup,
AggregatorFactory[] metricAggs,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException;
@VisibleForTesting
File merge(
List<IndexableAdapter> indexes,
boolean rollup,
@ -208,11 +225,21 @@ public interface IndexMerger
// Faster than IndexMaker
File convert(File inDir, File outDir, IndexSpec indexSpec) throws IOException;
File convert(File inDir, File outDir, IndexSpec indexSpec, ProgressIndicator progress)
throws IOException;
File convert(
File inDir,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException;
File append(List<IndexableAdapter> indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec)
throws IOException;
File append(
List<IndexableAdapter> indexes,
AggregatorFactory[] aggregators,
File outDir,
IndexSpec indexSpec,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException;
interface IndexSeeker
{
@ -385,7 +412,8 @@ public interface IndexMerger
Int2ObjectMap.Entry<IntSortedSet> 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);
}
}
}

View File

@ -28,17 +28,16 @@ import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Ordering;
import com.google.common.collect.Sets;
import com.google.common.io.ByteStreams;
import com.google.common.io.Files;
import com.google.common.primitives.Ints;
import com.google.common.primitives.Longs;
import com.google.inject.Inject;
import io.druid.collections.CombiningIterable;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.JodaUtils;
import io.druid.io.ZeroCopyByteArrayOutputStream;
import io.druid.java.util.common.DateTimes;
import io.druid.java.util.common.IAE;
import io.druid.java.util.common.ISE;
import io.druid.java.util.common.JodaUtils;
import io.druid.java.util.common.guava.Comparators;
import io.druid.java.util.common.guava.FunctionalIterable;
import io.druid.java.util.common.guava.MergeIterable;
@ -52,12 +51,10 @@ import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.ColumnCapabilitiesImpl;
import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.TmpFileIOPeon;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexAdapter;
import io.druid.segment.loading.MMappedQueryableSegmentizerFactory;
@ -67,6 +64,8 @@ import io.druid.segment.serde.ComplexMetrics;
import io.druid.segment.serde.DoubleGenericColumnPartSerde;
import io.druid.segment.serde.FloatGenericColumnPartSerde;
import io.druid.segment.serde.LongGenericColumnPartSerde;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.writeout.SegmentWriteOutMediumFactory;
import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
import it.unimi.dsi.fastutil.ints.IntIterator;
import it.unimi.dsi.fastutil.ints.IntSortedSet;
@ -75,7 +74,6 @@ import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.Closeable;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
@ -91,30 +89,18 @@ import java.util.Set;
public class IndexMergerV9 implements IndexMerger
{
private static final Logger log = new Logger(IndexMergerV9.class);
protected final ObjectMapper mapper;
protected final IndexIO indexIO;
private final ObjectMapper mapper;
private final IndexIO indexIO;
private final SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory;
@Inject
public IndexMergerV9(
ObjectMapper mapper,
IndexIO indexIO
)
public IndexMergerV9(ObjectMapper mapper, IndexIO indexIO, SegmentWriteOutMediumFactory defaultSegmentWriteOutMediumFactory)
{
this.mapper = Preconditions.checkNotNull(mapper, "null ObjectMapper");
this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
}
private static void registerDeleteDirectory(Closer closer, final File dir)
{
closer.register(new Closeable()
{
@Override
public void close() throws IOException
{
FileUtils.deleteDirectory(dir);
}
});
this.defaultSegmentWriteOutMediumFactory =
Preconditions.checkNotNull(defaultSegmentWriteOutMediumFactory, "null SegmentWriteOutMediumFactory");
}
private File makeIndexFiles(
@ -125,7 +111,8 @@ public class IndexMergerV9 implements IndexMerger
final List<String> mergedDimensions,
final List<String> mergedMetrics,
final Function<ArrayList<Iterable<Rowboat>>, Iterable<Rowboat>> rowMergerFn,
final IndexSpec indexSpec
final IndexSpec indexSpec,
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
progress.start();
@ -163,21 +150,15 @@ public class IndexMergerV9 implements IndexMerger
Closer closer = Closer.create();
try {
final FileSmoosher v9Smoosher = new FileSmoosher(outDir);
final File v9TmpDir = new File(outDir, "v9-tmp");
FileUtils.forceMkdir(v9TmpDir);
registerDeleteDirectory(closer, v9TmpDir);
log.info("Start making v9 index files, outDir:%s", outDir);
FileUtils.forceMkdir(outDir);
File tmpPeonFilesDir = new File(v9TmpDir, "tmpPeonFiles");
FileUtils.forceMkdir(tmpPeonFilesDir);
registerDeleteDirectory(closer, tmpPeonFilesDir);
final IOPeon ioPeon = new TmpFileIOPeon(tmpPeonFilesDir, false);
closer.register(ioPeon);
SegmentWriteOutMediumFactory omf = segmentWriteOutMediumFactory != null ? segmentWriteOutMediumFactory
: defaultSegmentWriteOutMediumFactory;
log.info("Using SegmentWriteOutMediumFactory[%s]", omf.getClass().getSimpleName());
SegmentWriteOutMedium segmentWriteOutMedium = omf.makeSegmentWriteOutMedium(outDir);
closer.register(segmentWriteOutMedium);
long startTime = System.currentTimeMillis();
ByteStreams.write(
Ints.toByteArray(IndexIO.V9_VERSION),
Files.newOutputStreamSupplier(new File(outDir, "version.bin"))
);
Files.asByteSink(new File(outDir, "version.bin")).write(Ints.toByteArray(IndexIO.V9_VERSION));
log.info("Completed version.bin in %,d millis.", System.currentTimeMillis() - startTime);
progress.progress();
@ -196,7 +177,7 @@ public class IndexMergerV9 implements IndexMerger
final DimensionHandler[] handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities);
final List<DimensionMerger> mergers = new ArrayList<>();
for (int i = 0; i < mergedDimensions.size(); i++) {
mergers.add(handlers[i].makeMerger(indexSpec, v9TmpDir, ioPeon, dimCapabilities.get(i), progress));
mergers.add(handlers[i].makeMerger(indexSpec, segmentWriteOutMedium, dimCapabilities.get(i), progress));
}
/************* Setup Dim Conversions **************/
@ -215,15 +196,17 @@ public class IndexMergerV9 implements IndexMerger
handlers,
mergers
);
final LongColumnSerializer timeWriter = setupTimeWriter(ioPeon, indexSpec);
final LongColumnSerializer timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec);
final ArrayList<GenericColumnSerializer> metWriters = setupMetricsWriters(
ioPeon, mergedMetrics, metricsValueTypes, metricTypeNames, indexSpec
segmentWriteOutMedium,
mergedMetrics,
metricsValueTypes,
metricTypeNames,
indexSpec
);
final List<IntBuffer> rowNumConversions = Lists.newArrayListWithCapacity(adapters.size());
mergeIndexesAndWriteColumns(
adapters, progress, theRows, timeWriter, metWriters, rowNumConversions, mergers
);
mergeIndexesAndWriteColumns(adapters, progress, theRows, timeWriter, metWriters, rowNumConversions, mergers);
/************ Create Inverted Indexes and Finalize Build Columns *************/
final String section = "build inverted index and columns";
@ -245,9 +228,7 @@ public class IndexMergerV9 implements IndexMerger
/************* Make index.drd & metadata.drd files **************/
progress.progress();
makeIndexBinary(
v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, progress, indexSpec, mergers
);
makeIndexBinary(v9Smoosher, adapters, outDir, mergedDimensions, mergedMetrics, progress, indexSpec, mergers);
makeMetadataBinary(v9Smoosher, progress, segmentMetadata);
v9Smoosher.close();
@ -312,8 +293,8 @@ public class IndexMergerV9 implements IndexMerger
+ serializerUtils.getSerializedStringByteSize(bitmapSerdeFactoryType);
final SmooshedWriter writer = v9Smoosher.addWithSmooshedWriter("index.drd", numBytes);
cols.writeToChannel(writer);
dims.writeToChannel(writer);
cols.writeTo(writer, v9Smoosher);
dims.writeTo(writer, v9Smoosher);
DateTime minTime = DateTimes.MAX;
DateTime maxTime = DateTimes.MIN;
@ -327,9 +308,7 @@ public class IndexMergerV9 implements IndexMerger
serializerUtils.writeLong(writer, dataInterval.getStartMillis());
serializerUtils.writeLong(writer, dataInterval.getEndMillis());
serializerUtils.writeString(
writer, bitmapSerdeFactoryType
);
serializerUtils.writeString(writer, bitmapSerdeFactoryType);
writer.close();
IndexIO.checkFileSize(new File(outDir, "index.drd"));
@ -355,7 +334,6 @@ public class IndexMergerV9 implements IndexMerger
String metric = mergedMetrics.get(i);
long metricStartTime = System.currentTimeMillis();
GenericColumnSerializer writer = metWriters.get(i);
writer.close();
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
ValueType type = metricsValueTypes.get(metric);
@ -421,8 +399,6 @@ public class IndexMergerV9 implements IndexMerger
progress.startSection(section);
long startTime = System.currentTimeMillis();
timeWriter.close();
final ColumnDescriptor serdeficator = ColumnDescriptor
.builder()
.setValueType(ValueType.LONG)
@ -447,10 +423,11 @@ public class IndexMergerV9 implements IndexMerger
ZeroCopyByteArrayOutputStream specBytes = new ZeroCopyByteArrayOutputStream();
serializerUtils.writeString(specBytes, mapper.writeValueAsString(serdeficator));
try (SmooshedWriter channel = v9Smoosher.addWithSmooshedWriter(
columnName, serdeficator.numBytes() + specBytes.size()
columnName,
specBytes.size() + serdeficator.getSerializedSize()
)) {
specBytes.writeTo(channel);
serdeficator.write(channel, v9Smoosher);
serdeficator.writeTo(channel, v9Smoosher);
}
}
@ -521,10 +498,12 @@ public class IndexMergerV9 implements IndexMerger
progress.stopSection(section);
}
private LongColumnSerializer setupTimeWriter(final IOPeon ioPeon, final IndexSpec indexSpec) throws IOException
private LongColumnSerializer setupTimeWriter(SegmentWriteOutMedium segmentWriteOutMedium, IndexSpec indexSpec) throws IOException
{
LongColumnSerializer timeWriter = LongColumnSerializer.create(
ioPeon, "little_end_time", CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY,
segmentWriteOutMedium,
"little_end_time",
CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY,
indexSpec.getLongEncoding()
);
// we will close this writer after we added all the timestamps
@ -533,7 +512,7 @@ public class IndexMergerV9 implements IndexMerger
}
private ArrayList<GenericColumnSerializer> setupMetricsWriters(
final IOPeon ioPeon,
final SegmentWriteOutMedium segmentWriteOutMedium,
final List<String> mergedMetrics,
final Map<String, ValueType> metricsValueTypes,
final Map<String, String> metricTypeNames,
@ -541,20 +520,20 @@ public class IndexMergerV9 implements IndexMerger
) throws IOException
{
ArrayList<GenericColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression();
final CompressionStrategy metCompression = indexSpec.getMetricCompression();
final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding();
for (String metric : mergedMetrics) {
ValueType type = metricsValueTypes.get(metric);
GenericColumnSerializer writer;
switch (type) {
case LONG:
writer = LongColumnSerializer.create(ioPeon, metric, metCompression, longEncoding);
writer = LongColumnSerializer.create(segmentWriteOutMedium, metric, metCompression, longEncoding);
break;
case FLOAT:
writer = FloatColumnSerializer.create(ioPeon, metric, metCompression);
writer = FloatColumnSerializer.create(segmentWriteOutMedium, metric, metCompression);
break;
case DOUBLE:
writer = DoubleColumnSerializer.create(ioPeon, metric, metCompression);
writer = DoubleColumnSerializer.create(segmentWriteOutMedium, metric, metCompression);
break;
case COMPLEX:
final String typeName = metricTypeNames.get(metric);
@ -562,7 +541,7 @@ public class IndexMergerV9 implements IndexMerger
if (serde == null) {
throw new ISE("Unknown type[%s]", typeName);
}
writer = serde.getSerializer(ioPeon, metric);
writer = serde.getSerializer(segmentWriteOutMedium, metric);
break;
default:
throw new ISE("Unknown type[%s]", type);
@ -628,21 +607,11 @@ public class IndexMergerV9 implements IndexMerger
public File persist(
final IncrementalIndex index,
File outDir,
IndexSpec indexSpec
IndexSpec indexSpec,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
return persist(index, index.getInterval(), outDir, indexSpec);
}
@Override
public File persist(
final IncrementalIndex index,
final Interval dataInterval,
File outDir,
IndexSpec indexSpec
) throws IOException
{
return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator());
return persist(index, index.getInterval(), outDir, indexSpec, segmentWriteOutMediumFactory);
}
@Override
@ -651,7 +620,20 @@ public class IndexMergerV9 implements IndexMerger
final Interval dataInterval,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
return persist(index, dataInterval, outDir, indexSpec, new BaseProgressIndicator(), segmentWriteOutMediumFactory);
}
@Override
public File persist(
final IncrementalIndex index,
final Interval dataInterval,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
if (index.isEmpty()) {
@ -688,7 +670,8 @@ public class IndexMergerV9 implements IndexMerger
index.getMetricAggs(),
outDir,
indexSpec,
progress
progress,
segmentWriteOutMediumFactory
);
}
@ -698,10 +681,19 @@ public class IndexMergerV9 implements IndexMerger
boolean rollup,
final AggregatorFactory[] metricAggs,
File outDir,
IndexSpec indexSpec
IndexSpec indexSpec,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
return mergeQueryableIndex(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator());
return mergeQueryableIndex(
indexes,
rollup,
metricAggs,
outDir,
indexSpec,
new BaseProgressIndicator(),
segmentWriteOutMediumFactory
);
}
@Override
@ -711,7 +703,8 @@ public class IndexMergerV9 implements IndexMerger
final AggregatorFactory[] metricAggs,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress
ProgressIndicator progress,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
return merge(
@ -720,7 +713,8 @@ public class IndexMergerV9 implements IndexMerger
metricAggs,
outDir,
indexSpec,
progress
progress,
segmentWriteOutMediumFactory
);
}
@ -733,16 +727,17 @@ public class IndexMergerV9 implements IndexMerger
IndexSpec indexSpec
) throws IOException
{
return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator());
return merge(indexes, rollup, metricAggs, outDir, indexSpec, new BaseProgressIndicator(), null);
}
public File merge(
private File merge(
List<IndexableAdapter> indexes,
final boolean rollup,
final AggregatorFactory[] metricAggs,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress
ProgressIndicator progress,
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
FileUtils.deleteDirectory(outDir);
@ -847,19 +842,25 @@ public class IndexMergerV9 implements IndexMerger
mergedDimensions,
mergedMetrics,
rowMergerFn,
indexSpec
indexSpec,
segmentWriteOutMediumFactory
);
}
@Override
public File convert(final File inDir, final File outDir, final IndexSpec indexSpec) throws IOException
{
return convert(inDir, outDir, indexSpec, new BaseProgressIndicator());
return convert(inDir, outDir, indexSpec, new BaseProgressIndicator(), defaultSegmentWriteOutMediumFactory);
}
@Override
public File convert(final File inDir, final File outDir, final IndexSpec indexSpec, final ProgressIndicator progress)
throws IOException
public File convert(
final File inDir,
final File outDir,
final IndexSpec indexSpec,
final ProgressIndicator progress,
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
try (QueryableIndex index = indexIO.loadIndex(inDir)) {
final IndexableAdapter adapter = new QueryableIndexIndexableAdapter(index);
@ -879,25 +880,19 @@ public class IndexMergerV9 implements IndexMerger
return input.get(0);
}
},
indexSpec
indexSpec,
segmentWriteOutMediumFactory
);
}
}
@Override
public File append(
List<IndexableAdapter> indexes, AggregatorFactory[] aggregators, File outDir, IndexSpec indexSpec
) throws IOException
{
return append(indexes, aggregators, outDir, indexSpec, new BaseProgressIndicator());
}
public File append(
List<IndexableAdapter> indexes,
AggregatorFactory[] aggregators,
File outDir,
IndexSpec indexSpec,
ProgressIndicator progress
@Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
{
FileUtils.deleteDirectory(outDir);
@ -944,11 +939,12 @@ public class IndexMergerV9 implements IndexMerger
indexes,
aggregators,
outDir,
progress,
new BaseProgressIndicator(),
mergedDimensions,
mergedMetrics,
rowMergerFn,
indexSpec
indexSpec,
segmentWriteOutMediumFactory
);
}

View File

@ -25,8 +25,8 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.Sets;
import io.druid.segment.data.BitmapSerde;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.ConciseBitmapSerdeFactory;
import java.util.Arrays;
@ -41,16 +41,16 @@ import java.util.Set;
*/
public class IndexSpec
{
public static final CompressedObjectStrategy.CompressionStrategy DEFAULT_METRIC_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY;
public static final CompressedObjectStrategy.CompressionStrategy DEFAULT_DIMENSION_COMPRESSION = CompressedObjectStrategy.DEFAULT_COMPRESSION_STRATEGY;
public static final CompressionStrategy DEFAULT_METRIC_COMPRESSION = CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY;
public static final CompressionStrategy DEFAULT_DIMENSION_COMPRESSION = CompressionStrategy.DEFAULT_COMPRESSION_STRATEGY;
public static final CompressionFactory.LongEncodingStrategy DEFAULT_LONG_ENCODING = CompressionFactory.DEFAULT_LONG_ENCODING_STRATEGY;
private static final Set<CompressedObjectStrategy.CompressionStrategy> METRIC_COMPRESSION = Sets.newHashSet(
Arrays.asList(CompressedObjectStrategy.CompressionStrategy.values())
private static final Set<CompressionStrategy> METRIC_COMPRESSION = Sets.newHashSet(
Arrays.asList(CompressionStrategy.values())
);
private static final Set<CompressedObjectStrategy.CompressionStrategy> DIMENSION_COMPRESSION = Sets.newHashSet(
Arrays.asList(CompressedObjectStrategy.CompressionStrategy.noNoneValues())
private static final Set<CompressionStrategy> DIMENSION_COMPRESSION = Sets.newHashSet(
Arrays.asList(CompressionStrategy.noNoneValues())
);
private static final Set<CompressionFactory.LongEncodingStrategy> LONG_ENCODING_NAMES = Sets.newHashSet(
@ -58,8 +58,8 @@ public class IndexSpec
);
private final BitmapSerdeFactory bitmapSerdeFactory;
private final CompressedObjectStrategy.CompressionStrategy dimensionCompression;
private final CompressedObjectStrategy.CompressionStrategy metricCompression;
private final CompressionStrategy dimensionCompression;
private final CompressionStrategy metricCompression;
private final CompressionFactory.LongEncodingStrategy longEncoding;
@ -80,10 +80,10 @@ public class IndexSpec
* setting, or, if none was set, uses the default defined in {@link BitmapSerde}
*
* @param dimensionCompression compression format for dimension columns, null to use the default.
* Defaults to {@link CompressedObjectStrategy#DEFAULT_COMPRESSION_STRATEGY}
* Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY}
*
* @param metricCompression compression format for metric columns, null to use the default.
* Defaults to {@link CompressedObjectStrategy#DEFAULT_COMPRESSION_STRATEGY}
* Defaults to {@link CompressionStrategy#DEFAULT_COMPRESSION_STRATEGY}
*
* @param longEncoding encoding strategy for metric and dimension columns with type long, null to use the default.
* Defaults to {@link CompressionFactory#DEFAULT_LONG_ENCODING_STRATEGY}
@ -91,8 +91,8 @@ public class IndexSpec
@JsonCreator
public IndexSpec(
@JsonProperty("bitmap") BitmapSerdeFactory bitmapSerdeFactory,
@JsonProperty("dimensionCompression") CompressedObjectStrategy.CompressionStrategy dimensionCompression,
@JsonProperty("metricCompression") CompressedObjectStrategy.CompressionStrategy metricCompression,
@JsonProperty("dimensionCompression") CompressionStrategy dimensionCompression,
@JsonProperty("metricCompression") CompressionStrategy metricCompression,
@JsonProperty("longEncoding") CompressionFactory.LongEncodingStrategy longEncoding
)
{
@ -118,13 +118,13 @@ public class IndexSpec
}
@JsonProperty
public CompressedObjectStrategy.CompressionStrategy getDimensionCompression()
public CompressionStrategy getDimensionCompression()
{
return dimensionCompression;
}
@JsonProperty
public CompressedObjectStrategy.CompressionStrategy getMetricCompression()
public CompressionStrategy getMetricCompression()
{
return metricCompression;
}

View File

@ -28,9 +28,6 @@ package io.druid.segment;
*/
public interface LongColumnSelector extends ColumnValueSelector<Long>
{
@Override
long getLong();
/**
* @deprecated This method is marked as deprecated in LongColumnSelector to minimize the probability of accidential
* calling. "Polymorphism" of LongColumnSelector should be used only when operating on {@link ColumnValueSelector}

View File

@ -21,9 +21,9 @@ package io.druid.segment;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.data.LongSupplierSerializer;
import java.io.IOException;
@ -36,31 +36,31 @@ import java.nio.channels.WritableByteChannel;
public class LongColumnSerializer implements GenericColumnSerializer
{
public static LongColumnSerializer create(
IOPeon ioPeon,
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
CompressedObjectStrategy.CompressionStrategy compression,
CompressionStrategy compression,
CompressionFactory.LongEncodingStrategy encoding
)
{
return new LongColumnSerializer(ioPeon, filenameBase, IndexIO.BYTE_ORDER, compression, encoding);
return new LongColumnSerializer(segmentWriteOutMedium, filenameBase, IndexIO.BYTE_ORDER, compression, encoding);
}
private final IOPeon ioPeon;
private final SegmentWriteOutMedium segmentWriteOutMedium;
private final String filenameBase;
private final ByteOrder byteOrder;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final CompressionStrategy compression;
private final CompressionFactory.LongEncodingStrategy encoding;
private LongSupplierSerializer writer;
public LongColumnSerializer(
IOPeon ioPeon,
private LongColumnSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
ByteOrder byteOrder,
CompressedObjectStrategy.CompressionStrategy compression,
CompressionStrategy compression,
CompressionFactory.LongEncodingStrategy encoding
)
{
this.ioPeon = ioPeon;
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.filenameBase = filenameBase;
this.byteOrder = byteOrder;
this.compression = compression;
@ -71,7 +71,7 @@ public class LongColumnSerializer implements GenericColumnSerializer
public void open() throws IOException
{
writer = CompressionFactory.getLongSerializer(
ioPeon,
segmentWriteOutMedium,
StringUtils.format("%s.long_column", filenameBase),
byteOrder,
encoding,
@ -88,21 +88,14 @@ public class LongColumnSerializer implements GenericColumnSerializer
}
@Override
public void close() throws IOException
{
writer.close();
}
@Override
public long getSerializedSize()
public long getSerializedSize() throws IOException
{
return writer.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
writer.writeToChannel(channel, smoosher);
writer.writeTo(channel, smoosher);
}
}

View File

@ -22,11 +22,10 @@ package io.druid.segment;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.GenericColumn;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.Indexed;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.Closeable;
import java.io.File;
import java.io.IOException;
public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
@ -52,13 +51,16 @@ public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
@Override
public DimensionMergerV9<Long> makeMerger(
IndexSpec indexSpec, File outDir, IOPeon ioPeon, ColumnCapabilities capabilities, ProgressIndicator progress
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
ProgressIndicator progress
) throws IOException
{
return new LongDimensionMergerV9(
dimensionName,
indexSpec,
ioPeon
segmentWriteOutMedium
);
}

View File

@ -22,10 +22,10 @@ package io.druid.segment;
import com.google.common.base.Throwables;
import io.druid.segment.column.ColumnDescriptor;
import io.druid.segment.column.ValueType;
import io.druid.segment.data.CompressedObjectStrategy;
import io.druid.segment.data.CompressionFactory;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.CompressionStrategy;
import io.druid.segment.serde.LongGenericColumnPartSerde;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
import java.nio.IntBuffer;
@ -35,32 +35,30 @@ public class LongDimensionMergerV9 implements DimensionMergerV9<Long>
{
protected String dimensionName;
protected final IndexSpec indexSpec;
protected IOPeon ioPeon;
protected LongColumnSerializer serializer;
public LongDimensionMergerV9(
LongDimensionMergerV9(
String dimensionName,
IndexSpec indexSpec,
IOPeon ioPeon
SegmentWriteOutMedium segmentWriteOutMedium
)
{
this.dimensionName = dimensionName;
this.indexSpec = indexSpec;
this.ioPeon = ioPeon;
try {
setupEncodedValueWriter();
setupEncodedValueWriter(segmentWriteOutMedium);
}
catch (IOException ioe) {
Throwables.propagate(ioe);
}
}
protected void setupEncodedValueWriter() throws IOException
protected void setupEncodedValueWriter(SegmentWriteOutMedium segmentWriteOutMedium) throws IOException
{
final CompressedObjectStrategy.CompressionStrategy metCompression = indexSpec.getMetricCompression();
final CompressionStrategy metCompression = indexSpec.getMetricCompression();
final CompressionFactory.LongEncodingStrategy longEncoding = indexSpec.getLongEncoding();
this.serializer = LongColumnSerializer.create(ioPeon, dimensionName, metCompression, longEncoding);
this.serializer = LongColumnSerializer.create(segmentWriteOutMedium, dimensionName, metCompression, longEncoding);
serializer.open();
}
@ -98,7 +96,6 @@ public class LongDimensionMergerV9 implements DimensionMergerV9<Long>
@Override
public ColumnDescriptor makeColumnDescriptor() throws IOException
{
serializer.close();
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
builder.setValueType(ValueType.LONG);
builder.addSerde(

View File

@ -40,7 +40,6 @@ public class MetricHolder
private static final byte[] version = new byte[]{0x0};
private static final SerializerUtils serializerUtils = new SerializerUtils();
public static MetricHolder fromByteBuffer(ByteBuffer buf, SmooshedFileMapper mapper) throws IOException
{
return fromByteBuffer(buf, null, mapper);
@ -60,7 +59,7 @@ public class MetricHolder
switch (holder.type) {
case FLOAT:
holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder(), mapper);
holder.floatType = CompressedFloatsIndexedSupplier.fromByteBuffer(buf, ByteOrder.nativeOrder());
break;
case COMPLEX:
if (strategy != null) {

View File

@ -51,7 +51,7 @@ public class Rowboat implements Comparable<Rowboat>
this.rowNum = rowNum;
this.handlers = handlers;
this.comprisedRows = new Int2ObjectOpenHashMap<>();
this.comprisedRows = new Int2ObjectOpenHashMap<>(1);
}
public long getTimestamp()

View File

@ -21,15 +21,14 @@ package io.druid.segment;
import com.google.common.primitives.Ints;
import io.druid.data.input.impl.DimensionSchema.MultiValueHandling;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.column.DictionaryEncodedColumn;
import io.druid.segment.data.IOPeon;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts;
import java.io.Closeable;
import java.io.File;
import java.lang.reflect.Array;
import java.util.Arrays;
@ -199,13 +198,12 @@ public class StringDimensionHandler implements DimensionHandler<Integer, int[],
@Override
public DimensionMergerV9 makeMerger(
IndexSpec indexSpec,
File outDir,
IOPeon ioPeon,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
ProgressIndicator progress
)
{
return new StringDimensionMergerV9(dimensionName, indexSpec, outDir, ioPeon, capabilities, progress);
return new StringDimensionMergerV9(dimensionName, indexSpec, segmentWriteOutMedium, capabilities, progress);
}
}

View File

@ -20,17 +20,14 @@
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.logger.Logger;
@ -41,28 +38,24 @@ import io.druid.segment.data.ArrayIndexed;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.BitmapValues;
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.ImmutableRTreeObjectStrategy;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedIntsWriter;
import io.druid.segment.data.VSizeIndexedIntsWriter;
import io.druid.segment.data.VSizeIndexedWriter;
import io.druid.segment.serde.DictionaryEncodedColumnPartSerde;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import it.unimi.dsi.fastutil.ints.IntIterable;
import it.unimi.dsi.fastutil.ints.IntIterator;
import javax.annotation.Nonnull;
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;
@ -70,35 +63,35 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
{
private static final Logger log = new Logger(StringDimensionMergerV9.class);
protected static final Indexed<String> 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<String> 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<String> dictionaryWriter;
protected GenericIndexedWriter<ImmutableBitmap> bitmapWriter;
protected ByteBufferWriter<ImmutableRTree> spatialWriter;
protected ArrayList<IntBuffer> dimConversions;
protected int cardinality = 0;
protected boolean convertMissingValues = false;
protected boolean hasNull = false;
protected MutableBitmap nullRowsBitmap;
protected IOPeon ioPeon;
protected int rowCount = 0;
protected ColumnCapabilities capabilities;
protected final File outDir;
protected List<IndexableAdapter> adapters;
protected ProgressIndicator progress;
protected final IndexSpec indexSpec;
protected IndexMerger.DictionaryMergeIterator dictionaryMergeIterator;
private String dimensionName;
private GenericIndexedWriter<String> dictionaryWriter;
private String firstDictionaryValue;
private int dictionarySize;
private GenericIndexedWriter<ImmutableBitmap> bitmapWriter;
private ByteBufferWriter<ImmutableRTree> spatialWriter;
private ArrayList<IntBuffer> dimConversions;
private int cardinality = 0;
private boolean convertMissingValues = false;
private boolean hasNull = false;
private MutableBitmap nullRowsBitmap;
private final SegmentWriteOutMedium segmentWriteOutMedium;
private int rowCount = 0;
private ColumnCapabilities capabilities;
private List<IndexableAdapter> adapters;
private ProgressIndicator progress;
private final IndexSpec indexSpec;
private IndexMerger.DictionaryMergeIterator dictionaryMergeIterator;
public StringDimensionMergerV9(
String dimensionName,
IndexSpec indexSpec,
File outDir,
IOPeon ioPeon,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
ProgressIndicator progress
)
@ -106,8 +99,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
this.dimensionName = dimensionName;
this.indexSpec = indexSpec;
this.capabilities = capabilities;
this.outDir = outDir;
this.ioPeon = ioPeon;
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.progress = progress;
nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
}
@ -158,21 +150,15 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
}
String dictFilename = StringUtils.format("%s.dim_values", dimensionName);
dictionaryWriter = new GenericIndexedWriter<>(
ioPeon,
dictFilename,
GenericIndexed.STRING_STRATEGY
);
dictionaryWriter = new GenericIndexedWriter<>(segmentWriteOutMedium, dictFilename, GenericIndexed.STRING_STRATEGY);
firstDictionaryValue = null;
dictionarySize = 0;
dictionaryWriter.open();
cardinality = 0;
if (numMergeIndex > 1) {
dictionaryMergeIterator = new IndexMerger.DictionaryMergeIterator(dimValueLookups, true);
while (dictionaryMergeIterator.hasNext()) {
dictionaryWriter.write(dictionaryMergeIterator.next());
}
writeDictionary(() -> dictionaryMergeIterator);
for (int i = 0; i < adapters.size(); i++) {
if (dimValueLookups[i] != null && dictionaryMergeIterator.needConversion(i)) {
dimConversions.set(i, dictionaryMergeIterator.conversions[i]);
@ -180,9 +166,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
}
cardinality = dictionaryMergeIterator.counter;
} else if (numMergeIndex == 1) {
for (String value : dimValueLookup) {
dictionaryWriter.write(value);
}
writeDictionary(dimValueLookup);
cardinality = dimValueLookup.size();
}
@ -192,34 +176,49 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
cardinality,
System.currentTimeMillis() - dimStartTime
);
dictionaryWriter.close();
setupEncodedValueWriter();
}
private void writeDictionary(Iterable<String> dictionaryValues) throws IOException
{
for (String value : dictionaryValues) {
dictionaryWriter.write(value);
value = Strings.emptyToNull(value);
if (dictionarySize == 0) {
firstDictionaryValue = value;
}
dictionarySize++;
}
}
protected void setupEncodedValueWriter() throws IOException
{
final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
String filenameBase = StringUtils.format("%s.forward_dim", dimensionName);
if (capabilities.hasMultipleValues()) {
encodedValueWriter = (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED)
? CompressedVSizeIndexedV3Writer.create(
ioPeon,
filenameBase,
cardinality,
compressionStrategy
)
: new VSizeIndexedWriter(ioPeon, filenameBase, cardinality);
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
encodedValueWriter = CompressedVSizeIndexedV3Writer.create(
segmentWriteOutMedium,
filenameBase,
cardinality,
compressionStrategy
);
} else {
encodedValueWriter = new VSizeIndexedWriter(segmentWriteOutMedium, cardinality);
}
} else {
encodedValueWriter = (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED)
? CompressedVSizeIntsIndexedWriter.create(
ioPeon,
filenameBase,
cardinality,
compressionStrategy
)
: new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality);
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
encodedValueWriter = CompressedVSizeIntsIndexedWriter.create(
segmentWriteOutMedium,
filenameBase,
cardinality,
compressionStrategy
);
} else {
encodedValueWriter = new VSizeIndexedIntsWriter(segmentWriteOutMedium, cardinality);
}
}
encodedValueWriter.open();
}
@ -277,87 +276,65 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
String bmpFilename = StringUtils.format("%s.inverted", dimensionName);
bitmapWriter = new GenericIndexedWriter<>(ioPeon, bmpFilename, bitmapSerdeFactory.getObjectStrategy());
bitmapWriter = new GenericIndexedWriter<>(
segmentWriteOutMedium,
bmpFilename,
indexSpec.getBitmapSerdeFactory().getObjectStrategy()
);
bitmapWriter.open();
bitmapWriter.setObjectsNotSorted();
// write dim values to one single file because we need to read it
File dimValueFile = IndexIO.makeDimFile(outDir, dimensionName);
try (FileOutputStream fos = new FileOutputStream(dimValueFile)) {
ByteStreams.copy(dictionaryWriter.combineStreams(), fos);
BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
RTree tree = null;
boolean hasSpatial = capabilities.hasSpatialIndexes();
if (hasSpatial) {
spatialWriter = new ByteBufferWriter<>(
segmentWriteOutMedium,
new ImmutableRTreeObjectStrategy(bitmapFactory)
);
spatialWriter.open();
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory);
}
final MappedByteBuffer dimValsMapped = Files.map(dimValueFile);
try (
Closeable toCloseEncodedValueWriter = encodedValueWriter;
Closeable toCloseBitmapWriter = bitmapWriter;
// We need to free the ByteBuffers allocated by the dictionary merge iterator here,
// these buffers are used by dictIdSeeker in mergeBitmaps() below. The iterator is created and only used
// in writeMergedValueMetadata(), but the buffers are still used until after mergeBitmaps().
Closeable toCloseDictionaryMergeIterator = dictionaryMergeIterator;
Closeable dimValsMappedUnmapper = () -> ByteBufferUtils.unmap(dimValsMapped)
) {
Indexed<String> dimVals = GenericIndexed.read(dimValsMapped, GenericIndexed.STRING_STRATEGY);
BitmapFactory bmpFactory = bitmapSerdeFactory.getBitmapFactory();
IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName);
RTree tree = null;
boolean hasSpatial = capabilities.hasSpatialIndexes();
if (hasSpatial) {
spatialWriter = new ByteBufferWriter<>(
ioPeon,
StringUtils.format("%s.spatial", dimensionName),
new ImmutableRTreeObjectStrategy(bmpFactory)
);
spatialWriter.open();
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory);
}
IndexSeeker[] dictIdSeeker = toIndexSeekers(adapters, dimConversions, dimensionName);
//Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result.
for (int dictId = 0; dictId < dimVals.size(); dictId++) {
progress.progress();
mergeBitmaps(
segmentRowNumConversions,
dimVals,
bmpFactory,
tree,
hasSpatial,
dictIdSeeker,
dictId,
adapters,
dimensionName,
nullRowsBitmap,
bitmapWriter
);
}
if (hasSpatial) {
spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree));
spatialWriter.close();
}
log.info(
"Completed dim[%s] inverted with cardinality[%,d] in %,d millis.",
dimensionName,
dimVals.size(),
System.currentTimeMillis() - dimStartTime
//Iterate all dim values's dictionary id in ascending order which in line with dim values's compare result.
for (int dictId = 0; dictId < dictionarySize; dictId++) {
progress.progress();
mergeBitmaps(
segmentRowNumConversions,
bitmapFactory,
tree,
hasSpatial,
dictIdSeeker,
dictId
);
}
if (hasSpatial) {
spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree));
}
log.info(
"Completed dim[%s] inverted with cardinality[%,d] in %,d millis.",
dimensionName,
dictionarySize,
System.currentTimeMillis() - dimStartTime
);
if (dictionaryMergeIterator != null) {
dictionaryMergeIterator.close();
}
}
static void mergeBitmaps(
void mergeBitmaps(
List<IntBuffer> segmentRowNumConversions,
Indexed<String> dimVals,
BitmapFactory bmpFactory,
RTree tree,
boolean hasSpatial,
IndexSeeker[] dictIdSeeker,
int dictId,
List<IndexableAdapter> adapters,
String dimensionName,
MutableBitmap nullRowsBitmap,
GenericIndexedWriter<ImmutableBitmap> bitmapWriter
int dictId
) throws IOException
{
List<ConvertingBitmapValues> convertedInvertedIndexesToMerge = Lists.newArrayListWithCapacity(adapters.size());
@ -392,14 +369,14 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
prevRow = row;
}
if ((dictId == 0) && (Iterables.getFirst(dimVals, "") == null)) {
if (dictId == 0 && firstDictionaryValue == null) {
mergedIndexes.or(nullRowsBitmap);
}
bitmapWriter.write(bmpFactory.makeImmutableBitmap(mergedIndexes));
if (hasSpatial) {
String dimVal = dimVals.get(dictId);
String dimVal = dictionaryWriter.get(dictId);
if (dimVal != null) {
List<String> stringCoords = Lists.newArrayList(SPLITTER.split(dimVal));
float[] coords = new float[stringCoords.size()];
@ -422,7 +399,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
{
// Now write everything
boolean hasMultiValue = capabilities.hasMultipleValues();
final CompressedObjectStrategy.CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
final ColumnDescriptor.Builder builder = ColumnDescriptor.builder();
@ -434,7 +411,7 @@ public class StringDimensionMergerV9 implements DimensionMergerV9<int[]>
.withValue(
encodedValueWriter,
hasMultiValue,
compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED
compressionStrategy != CompressionStrategy.UNCOMPRESSED
)
.withBitmapSerdeFactory(bitmapSerdeFactory)
.withBitmapIndex(bitmapWriter)

View File

@ -27,6 +27,7 @@ import io.druid.java.util.common.IAE;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import io.druid.segment.serde.ColumnPartSerde;
import io.druid.segment.serde.Serializer;
import java.io.IOException;
import java.nio.ByteBuffer;
@ -35,7 +36,7 @@ import java.util.List;
/**
*/
public class ColumnDescriptor
public class ColumnDescriptor implements Serializer
{
public static Builder builder()
{
@ -76,21 +77,21 @@ public class ColumnDescriptor
return parts;
}
public long numBytes()
@Override
public long getSerializedSize() throws IOException
{
long retVal = 0;
long size = 0;
for (ColumnPartSerde part : parts) {
retVal += part.getSerializer().numBytes();
size += part.getSerializer().getSerializedSize();
}
return retVal;
return size;
}
public void write(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
@Override
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
for (ColumnPartSerde part : parts) {
part.getSerializer().write(channel, smoosher);
part.getSerializer().writeTo(channel, smoosher);
}
}

View File

@ -19,52 +19,49 @@
package io.druid.segment.data;
import com.google.common.io.ByteStreams;
import com.google.common.io.CountingOutputStream;
import com.google.common.primitives.Doubles;
import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
import io.druid.java.util.common.io.Closer;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.CompressedPools;
import io.druid.segment.serde.MetaSerdeHelper;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.DoubleBuffer;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
public class BlockLayoutDoubleSupplierSerializer implements DoubleSupplierSerializer
{
private final IOPeon ioPeon;
private final int sizePer;
private final GenericIndexedWriter<ResourceHolder<DoubleBuffer>> flattener;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final String metaFile;
private static final MetaSerdeHelper<BlockLayoutDoubleSupplierSerializer> 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<ByteBuffer> flattener;
private final CompressionStrategy compression;
private long metaCount = 0;
private int numInserted = 0;
private DoubleBuffer endBuffer;
private ByteBuffer endBuffer;
public BlockLayoutDoubleSupplierSerializer(
IOPeon ioPeon,
BlockLayoutDoubleSupplierSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
ByteOrder order,
CompressedObjectStrategy.CompressionStrategy compression
ByteOrder byteOrder,
CompressionStrategy compression
)
{
this.ioPeon = ioPeon;
this.sizePer = CompressedPools.BUFFER_SIZE / Doubles.BYTES;
this.flattener = new GenericIndexedWriter<>(
ioPeon, filenameBase, CompressedDoubleBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)
this.flattener = GenericIndexedWriter.ofCompressedByteBuffers(
segmentWriteOutMedium,
filenameBase,
compression,
CompressedPools.BUFFER_SIZE
);
this.metaFile = filenameBase + ".format";
this.compression = compression;
endBuffer = DoubleBuffer.allocate(sizePer);
endBuffer.mark();
CompressionStrategy.Compressor compressor = compression.getCompressor();
Closer closer = segmentWriteOutMedium.getCloser();
this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder);
}
@Override
@ -76,48 +73,42 @@ public class BlockLayoutDoubleSupplierSerializer implements DoubleSupplierSerial
@Override
public void add(double value) throws IOException
{
if (endBuffer == null) {
throw new IllegalStateException("written out already");
}
if (!endBuffer.hasRemaining()) {
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = DoubleBuffer.allocate(sizePer);
endBuffer.mark();
flattener.write(endBuffer);
endBuffer.clear();
}
endBuffer.put(value);
endBuffer.putDouble(value);
++numInserted;
}
@Override
public long getSerializedSize()
public long getSerializedSize() throws IOException
{
return metaCount + flattener.getSerializedSize();
writeEndBuffer();
return metaSerdeHelper.size(this) + flattener.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
try (InputStream meta = ioPeon.makeInputStream(metaFile)) {
ByteStreams.copy(Channels.newChannel(meta), channel);
flattener.writeToChannel(channel, smoosher);
}
writeEndBuffer();
metaSerdeHelper.writeTo(channel, this);
flattener.writeTo(channel, smoosher);
}
@Override
public void close() throws IOException
private void writeEndBuffer() throws IOException
{
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = null;
flattener.close();
try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) {
metaOut.write(CompressedDoublesIndexedSupplier.version);
metaOut.write(Ints.toByteArray(numInserted));
metaOut.write(Ints.toByteArray(sizePer));
metaOut.write(compression.getId());
metaOut.close();
metaCount = metaOut.getCount();
if (endBuffer != null) {
endBuffer.flip();
if (endBuffer.remaining() > 0) {
flattener.write(endBuffer);
}
endBuffer = null;
}
}
}

View File

@ -19,53 +19,48 @@
package io.druid.segment.data;
import com.google.common.io.ByteSink;
import com.google.common.io.ByteStreams;
import com.google.common.io.CountingOutputStream;
import com.google.common.primitives.Floats;
import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
import io.druid.java.util.common.io.Closer;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.CompressedPools;
import io.druid.segment.serde.MetaSerdeHelper;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.FloatBuffer;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
public class BlockLayoutFloatSupplierSerializer implements FloatSupplierSerializer
{
private final IOPeon ioPeon;
private final int sizePer;
private final GenericIndexedWriter<ResourceHolder<FloatBuffer>> flattener;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final String metaFile;
private static final MetaSerdeHelper<BlockLayoutFloatSupplierSerializer> 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<ByteBuffer> flattener;
private final CompressionStrategy compression;
private long metaCount = 0;
private int numInserted = 0;
private FloatBuffer endBuffer;
private ByteBuffer endBuffer;
public BlockLayoutFloatSupplierSerializer(
IOPeon ioPeon,
BlockLayoutFloatSupplierSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
ByteOrder order,
CompressedObjectStrategy.CompressionStrategy compression
ByteOrder byteOrder,
CompressionStrategy compression
)
{
this.ioPeon = ioPeon;
this.sizePer = CompressedPools.BUFFER_SIZE / Floats.BYTES;
this.flattener = new GenericIndexedWriter<>(
ioPeon, filenameBase, CompressedFloatBufferObjectStrategy.getBufferForOrder(order, compression, sizePer)
this.flattener = GenericIndexedWriter.ofCompressedByteBuffers(
segmentWriteOutMedium,
filenameBase,
compression,
CompressedPools.BUFFER_SIZE
);
this.metaFile = filenameBase + ".format";
this.compression = compression;
endBuffer = FloatBuffer.allocate(sizePer);
endBuffer.mark();
CompressionStrategy.Compressor compressor = compression.getCompressor();
Closer closer = segmentWriteOutMedium.getCloser();
this.endBuffer = compressor.allocateInBuffer(CompressedPools.BUFFER_SIZE, closer).order(byteOrder);
}
@Override
@ -83,59 +78,41 @@ public class BlockLayoutFloatSupplierSerializer implements FloatSupplierSerializ
@Override
public void add(float value) throws IOException
{
if (endBuffer == null) {
throw new IllegalStateException("written out already");
}
if (!endBuffer.hasRemaining()) {
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = FloatBuffer.allocate(sizePer);
endBuffer.mark();
flattener.write(endBuffer);
endBuffer.clear();
}
endBuffer.put(value);
endBuffer.putFloat(value);
++numInserted;
}
@Override
public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException
public long getSerializedSize() throws IOException
{
close();
try (OutputStream out = consolidatedOut.openStream();
InputStream meta = ioPeon.makeInputStream(metaFile)) {
ByteStreams.copy(meta, out);
ByteStreams.copy(flattener.combineStreams(), out);
}
writeEndBuffer();
return metaSerdeHelper.size(this) + flattener.getSerializedSize();
}
@Override
public void close() throws IOException
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
endBuffer = null;
flattener.close();
try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) {
metaOut.write(CompressedFloatsIndexedSupplier.version);
metaOut.write(Ints.toByteArray(numInserted));
metaOut.write(Ints.toByteArray(sizePer));
metaOut.write(compression.getId());
metaOut.close();
metaCount = metaOut.getCount();
}
writeEndBuffer();
metaSerdeHelper.writeTo(channel, this);
flattener.writeTo(channel, smoosher);
}
@Override
public long getSerializedSize()
private void writeEndBuffer() throws IOException
{
return metaCount + flattener.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
try (InputStream meta = ioPeon.makeInputStream(metaFile)) {
ByteStreams.copy(Channels.newChannel(meta), channel);
flattener.writeToChannel(channel, smoosher);
if (endBuffer != null) {
endBuffer.flip();
if (endBuffer.remaining() > 0) {
flattener.write(endBuffer);
}
endBuffer = null;
}
}
}

View File

@ -20,10 +20,8 @@
package io.druid.segment.data;
import com.google.common.base.Supplier;
import com.google.common.primitives.Doubles;
import io.druid.collections.ResourceHolder;
import io.druid.java.util.common.guava.CloseQuietly;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@ -41,18 +39,13 @@ public class BlockLayoutIndexedDoubleSupplier implements Supplier<IndexedDoubles
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;

View File

@ -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 class BlockLayoutIndexedFloatSupplier implements Supplier<IndexedFloats>
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;

View File

@ -23,7 +23,6 @@ import com.google.common.base.Supplier;
import io.druid.collections.ResourceHolder;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.CloseQuietly;
import io.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
@ -43,19 +42,10 @@ public class BlockLayoutIndexedLongSupplier implements Supplier<IndexedLongs>
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;

View File

@ -19,60 +19,50 @@
package io.druid.segment.data;
import com.google.common.io.ByteSink;
import com.google.common.io.ByteStreams;
import com.google.common.io.CountingOutputStream;
import com.google.common.primitives.Ints;
import io.druid.collections.ResourceHolder;
import io.druid.collections.StupidResourceHolder;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.CompressedPools;
import io.druid.segment.serde.MetaSerdeHelper;
import java.io.IOException;
import java.io.InputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.nio.channels.Channels;
import java.nio.channels.WritableByteChannel;
public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer
{
private static final MetaSerdeHelper<BlockLayoutLongSupplierSerializer> metaSerdeHelper = MetaSerdeHelper
.firstWriteByte((BlockLayoutLongSupplierSerializer x) -> CompressedLongsIndexedSupplier.VERSION)
.writeInt(x -> x.numInserted)
.writeInt(x -> x.sizePer)
.writeSomething(CompressionFactory.longEncodingWriter(x -> x.writer, x -> x.compression));
private final IOPeon ioPeon;
private final int sizePer;
private final CompressionFactory.LongEncodingWriter writer;
private final GenericIndexedWriter<ResourceHolder<ByteBuffer>> flattener;
private final CompressedObjectStrategy.CompressionStrategy compression;
private final String metaFile;
private long metaCount = 0;
private final GenericIndexedWriter<ByteBuffer> flattener;
private final CompressionStrategy compression;
private int numInserted = 0;
private int numInsertedForNextFlush;
private ByteBuffer endBuffer = null;
public BlockLayoutLongSupplierSerializer(
IOPeon ioPeon,
BlockLayoutLongSupplierSerializer(
SegmentWriteOutMedium segmentWriteOutMedium,
String filenameBase,
ByteOrder order,
ByteOrder byteOrder,
CompressionFactory.LongEncodingWriter writer,
CompressedObjectStrategy.CompressionStrategy compression
CompressionStrategy compression
)
{
this.ioPeon = ioPeon;
this.sizePer = writer.getBlockSize(CompressedPools.BUFFER_SIZE);
this.flattener = new GenericIndexedWriter<>(
ioPeon,
filenameBase,
VSizeCompressedObjectStrategy.getBufferForOrder(
order,
compression,
writer.getNumBytes(sizePer)
)
);
this.metaFile = filenameBase + ".format";
int bufferSize = writer.getNumBytes(sizePer);
this.flattener = GenericIndexedWriter.ofCompressedByteBuffers(segmentWriteOutMedium, filenameBase, compression, bufferSize);
this.writer = writer;
this.compression = compression;
CompressionStrategy.Compressor compressor = compression.getCompressor();
endBuffer = compressor.allocateInBuffer(writer.getNumBytes(sizePer), segmentWriteOutMedium.getCloser()).order(byteOrder);
writer.setBuffer(endBuffer);
numInsertedForNextFlush = sizePer;
}
@Override
@ -90,15 +80,15 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer
@Override
public void add(long value) throws IOException
{
if (numInserted % sizePer == 0) {
if (endBuffer != null) {
writer.flush();
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
}
endBuffer = ByteBuffer.allocate(writer.getNumBytes(sizePer));
writer.setBuffer(endBuffer);
if (endBuffer == null) {
throw new IllegalStateException("written out already");
}
if (numInserted == numInsertedForNextFlush) {
numInsertedForNextFlush += sizePer;
writer.flush();
endBuffer.flip();
flattener.write(endBuffer);
endBuffer.clear();
}
writer.write(value);
@ -106,50 +96,29 @@ public class BlockLayoutLongSupplierSerializer implements LongSupplierSerializer
}
@Override
public void closeAndConsolidate(ByteSink consolidatedOut) throws IOException
public long getSerializedSize() throws IOException
{
close();
try (OutputStream out = consolidatedOut.openStream();
InputStream meta = ioPeon.makeInputStream(metaFile)) {
ByteStreams.copy(meta, out);
ByteStreams.copy(flattener.combineStreams(), out);
}
writeEndBuffer();
return metaSerdeHelper.size(this) + flattener.getSerializedSize();
}
@Override
public void close() throws IOException
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
writeEndBuffer();
metaSerdeHelper.writeTo(channel, this);
flattener.writeTo(channel, smoosher);
}
private void writeEndBuffer() throws IOException
{
if (endBuffer != null) {
writer.flush();
endBuffer.limit(endBuffer.position());
endBuffer.rewind();
flattener.write(StupidResourceHolder.create(endBuffer));
}
endBuffer = null;
flattener.close();
try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) {
metaOut.write(CompressedLongsIndexedSupplier.version);
metaOut.write(Ints.toByteArray(numInserted));
metaOut.write(Ints.toByteArray(sizePer));
writer.putMeta(metaOut, compression);
metaOut.close();
metaCount = metaOut.getCount();
}
}
@Override
public long getSerializedSize()
{
return metaCount + flattener.getSerializedSize();
}
@Override
public void writeToChannel(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
try (InputStream meta = ioPeon.makeInputStream(metaFile)) {
ByteStreams.copy(Channels.newChannel(meta), channel);
flattener.writeToChannel(channel, smoosher);
endBuffer.flip();
if (endBuffer.remaining() > 0) {
flattener.write(endBuffer);
}
endBuffer = null;
}
}
}

View File

@ -23,7 +23,7 @@ import java.nio.ByteBuffer;
/**
*/
public class ByteBufferSerializer<T>
public class ByteBufferSerializer
{
public static <T> T read(ByteBuffer buffer, ObjectStrategy<T> strategy)
{
@ -34,5 +34,4 @@ public class ByteBufferSerializer<T>
return strategy.fromByteBuffer(bufferToUse, size);
}
}

View File

@ -19,111 +19,60 @@
package io.druid.segment.data;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.io.ByteStreams;
import com.google.common.io.CountingOutputStream;
import com.google.common.io.InputSupplier;
import com.google.common.primitives.Ints;
import io.druid.common.utils.SerializerUtils;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.io.smoosh.FileSmoosher;
import io.druid.segment.serde.Serializer;
import io.druid.segment.writeout.SegmentWriteOutMedium;
import io.druid.segment.writeout.WriteOutBytes;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.nio.channels.ReadableByteChannel;
import java.nio.channels.WritableByteChannel;
import java.util.Arrays;
/**
*/
public class ByteBufferWriter<T> implements Closeable
public class ByteBufferWriter<T> implements Serializer
{
private final IOPeon ioPeon;
private final String filenameBase;
private final SegmentWriteOutMedium segmentWriteOutMedium;
private final ObjectStrategy<T> strategy;
private CountingOutputStream headerOut = null;
private CountingOutputStream valueOut = null;
private final ByteBuffer helperBuffer = ByteBuffer.allocate(Ints.BYTES);
private WriteOutBytes headerOut = null;
private WriteOutBytes valueOut = null;
public ByteBufferWriter(
IOPeon ioPeon,
String filenameBase,
ObjectStrategy<T> strategy
)
public ByteBufferWriter(SegmentWriteOutMedium segmentWriteOutMedium, ObjectStrategy<T> strategy)
{
this.ioPeon = ioPeon;
this.filenameBase = filenameBase;
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.strategy = strategy;
}
public void open() throws IOException
{
headerOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("header")));
valueOut = new CountingOutputStream(ioPeon.makeOutputStream(makeFilename("value")));
headerOut = segmentWriteOutMedium.makeWriteOutBytes();
valueOut = segmentWriteOutMedium.makeWriteOutBytes();
}
public void write(T objectToWrite) throws IOException
{
byte[] bytesToWrite = strategy.toBytes(objectToWrite);
SerializerUtils.writeBigEndianIntToOutputStream(headerOut, bytesToWrite.length, helperBuffer);
valueOut.write(bytesToWrite);
}
private String makeFilename(String suffix)
{
return StringUtils.format("%s.%s", filenameBase, suffix);
long sizeBefore = valueOut.size();
strategy.writeTo(objectToWrite, valueOut);
headerOut.writeInt(Ints.checkedCast(valueOut.size() - sizeBefore));
}
@Override
public void close() throws IOException
public long getSerializedSize() throws IOException
{
headerOut.close();
valueOut.close();
return headerOut.size() + valueOut.size();
}
final long numBytesWritten = headerOut.getCount() + valueOut.getCount();
@Override
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
final long numBytesWritten = headerOut.size() + valueOut.size();
Preconditions.checkState(
numBytesWritten < Integer.MAX_VALUE, "Wrote[%s] bytes, which is too many.", numBytesWritten
);
}
public long getSerializedSize()
{
return headerOut.getCount() + valueOut.getCount();
}
public InputSupplier<InputStream> combineStreams()
{
return ByteStreams.join(
Iterables.transform(
Arrays.asList("header", "value"),
new Function<String, InputSupplier<InputStream>>()
{
@Override
public InputSupplier<InputStream> apply(final String input)
{
return new InputSupplier<InputStream>()
{
@Override
public InputStream getInput() throws IOException
{
return ioPeon.makeInputStream(makeFilename(input));
}
};
}
}
)
);
}
public void writeToChannel(WritableByteChannel channel) throws IOException
{
try (final ReadableByteChannel from = Channels.newChannel(combineStreams().getInput())) {
ByteStreams.copy(from, channel);
}
headerOut.writeTo(channel);
valueOut.writeTo(channel);
}
}

View File

@ -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<ByteBuffer>
{
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<ByteBuffer>()
{
@Override
public ByteBuffer convert(ByteBuffer buf)
{
return buf;
}
@Override
public int compare(ByteBuffer lhs, ByteBuffer rhs)
{
return Comparators.<ByteBuffer>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
);
}
}

View File

@ -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<DoubleBuffer>
{
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<DoubleBuffer>()
{
@Override
public DoubleBuffer convert(ByteBuffer buf)
{
return buf.asDoubleBuffer();
}
@Override
public int compare(DoubleBuffer lhs, DoubleBuffer rhs)
{
return Comparators.<DoubleBuffer>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);
}
}

Some files were not shown because too many files have changed in this diff Show More