mirror of https://github.com/apache/druid.git
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:
parent
7c01f77b04
commit
a7a6a0487e
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
);
|
||||
try (
|
||||
BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8);
|
||||
|
||||
try (FileChannel output = FileChannel.open(
|
||||
compFile.toPath(),
|
||||
StandardOpenOption.CREATE_NEW,
|
||||
StandardOpenOption.WRITE
|
||||
)) {
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
);
|
||||
try (
|
||||
BufferedReader br = Files.newBufferedReader(dataFile.toPath(), StandardCharsets.UTF_8);
|
||||
|
||||
try (FileChannel output = FileChannel.open(
|
||||
compFile.toPath(),
|
||||
StandardOpenOption.CREATE_NEW,
|
||||
StandardOpenOption.WRITE
|
||||
)) {
|
||||
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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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() {}
|
||||
}
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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()
|
||||
{
|
||||
}
|
||||
}
|
|
@ -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`|
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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`.
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
}
|
||||
|
|
|
@ -88,6 +88,7 @@ public class KafkaSupervisorSpec implements SupervisorSpec
|
|||
null,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
null
|
||||
);
|
||||
this.ioConfig = Preconditions.checkNotNull(ioConfig, "ioConfig");
|
||||
|
|
|
@ -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 +
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -111,6 +111,7 @@ public class KafkaTuningConfigTest
|
|||
true,
|
||||
true,
|
||||
5L,
|
||||
null,
|
||||
null
|
||||
);
|
||||
KafkaTuningConfig copy = KafkaTuningConfig.copyOf(original);
|
||||
|
|
|
@ -194,6 +194,7 @@ public class KafkaSupervisorTest extends EasyMockSupport
|
|||
false,
|
||||
null,
|
||||
null,
|
||||
null,
|
||||
numThreads,
|
||||
TEST_CHAT_THREADS,
|
||||
TEST_CHAT_RETRIES,
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -535,7 +535,8 @@ public class HadoopConverterJob
|
|||
inDir,
|
||||
outDir,
|
||||
config.getIndexSpec(),
|
||||
JobHelper.progressIndicatorForContext(context)
|
||||
JobHelper.progressIndicatorForContext(context),
|
||||
null
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,16 +410,8 @@ 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
|
||||
private void convertSegment(TaskToolbox toolbox) throws SegmentLoadingException, IOException
|
||||
{
|
||||
log.info("Converting segment[%s]", segment);
|
||||
final TaskActionClient actionClient = toolbox.getTaskActionClient();
|
||||
|
@ -391,7 +433,8 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
|
|||
|
||||
final File location = localSegments.get(segment);
|
||||
final File outLocation = new File(location, "v9_out");
|
||||
if (toolbox.getIndexIO().convertSegment(location, outLocation, indexSpec, force, validate)) {
|
||||
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
|
||||
|
@ -404,4 +447,5 @@ public class ConvertSegmentTask extends AbstractFixedIntervalTask
|
|||
log.info("Conversion failed.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
|
|
@ -1147,6 +1147,7 @@ public class IndexTaskTest
|
|||
forceExtendableShardSpecs,
|
||||
forceGuaranteedRollup,
|
||||
reportParseException,
|
||||
null,
|
||||
null
|
||||
);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -931,6 +931,7 @@ public class RealtimeIndexTaskTest
|
|||
0,
|
||||
reportParseExceptions,
|
||||
handoffTimeout,
|
||||
null,
|
||||
null
|
||||
);
|
||||
return new RealtimeIndexTask(
|
||||
|
|
|
@ -91,6 +91,7 @@ public class SameIntervalMergeTaskTest
|
|||
true,
|
||||
indexSpec,
|
||||
true,
|
||||
null,
|
||||
null
|
||||
);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
4
pom.xml
4
pom.xml
|
@ -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>
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
|
||||
encodedValueWriter = CompressedVSizeIndexedV3Writer.create(
|
||||
segmentWriteOutMedium,
|
||||
filenameBase,
|
||||
cardinality,
|
||||
compressionStrategy
|
||||
)
|
||||
: new VSizeIndexedWriter(ioPeon, filenameBase, cardinality);
|
||||
);
|
||||
} else {
|
||||
encodedValueWriter = (compressionStrategy != CompressedObjectStrategy.CompressionStrategy.UNCOMPRESSED)
|
||||
? CompressedVSizeIntsIndexedWriter.create(
|
||||
ioPeon,
|
||||
encodedValueWriter = new VSizeIndexedWriter(segmentWriteOutMedium, cardinality);
|
||||
}
|
||||
} else {
|
||||
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
|
||||
encodedValueWriter = CompressedVSizeIntsIndexedWriter.create(
|
||||
segmentWriteOutMedium,
|
||||
filenameBase,
|
||||
cardinality,
|
||||
compressionStrategy
|
||||
)
|
||||
: new VSizeIndexedIntsWriter(ioPeon, filenameBase, cardinality);
|
||||
);
|
||||
} 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);
|
||||
}
|
||||
|
||||
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();
|
||||
BitmapFactory bitmapFactory = bitmapSerdeFactory.getBitmapFactory();
|
||||
|
||||
RTree tree = null;
|
||||
boolean hasSpatial = capabilities.hasSpatialIndexes();
|
||||
if (hasSpatial) {
|
||||
spatialWriter = new ByteBufferWriter<>(
|
||||
ioPeon,
|
||||
StringUtils.format("%s.spatial", dimensionName),
|
||||
new ImmutableRTreeObjectStrategy(bmpFactory)
|
||||
segmentWriteOutMedium,
|
||||
new ImmutableRTreeObjectStrategy(bitmapFactory)
|
||||
);
|
||||
spatialWriter.open();
|
||||
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bmpFactory), bmpFactory);
|
||||
tree = new RTree(2, new LinearGutmanSplitStrategy(0, 50, bitmapFactory), bitmapFactory);
|
||||
}
|
||||
|
||||
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++) {
|
||||
for (int dictId = 0; dictId < dictionarySize; dictId++) {
|
||||
progress.progress();
|
||||
mergeBitmaps(
|
||||
segmentRowNumConversions,
|
||||
dimVals,
|
||||
bmpFactory,
|
||||
bitmapFactory,
|
||||
tree,
|
||||
hasSpatial,
|
||||
dictIdSeeker,
|
||||
dictId,
|
||||
adapters,
|
||||
dimensionName,
|
||||
nullRowsBitmap,
|
||||
bitmapWriter
|
||||
dictId
|
||||
);
|
||||
}
|
||||
|
||||
if (hasSpatial) {
|
||||
spatialWriter.write(ImmutableRTree.newImmutableFromMutable(tree));
|
||||
spatialWriter.close();
|
||||
}
|
||||
|
||||
log.info(
|
||||
"Completed dim[%s] inverted with cardinality[%,d] in %,d millis.",
|
||||
dimensionName,
|
||||
dimVals.size(),
|
||||
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)
|
||||
|
|
|
@ -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 size;
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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));
|
||||
if (endBuffer != null) {
|
||||
endBuffer.flip();
|
||||
if (endBuffer.remaining() > 0) {
|
||||
flattener.write(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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
writeEndBuffer();
|
||||
metaSerdeHelper.writeTo(channel, this);
|
||||
flattener.writeTo(channel, smoosher);
|
||||
}
|
||||
|
||||
private void writeEndBuffer() throws IOException
|
||||
{
|
||||
if (endBuffer != null) {
|
||||
endBuffer.flip();
|
||||
if (endBuffer.remaining() > 0) {
|
||||
flattener.write(endBuffer);
|
||||
}
|
||||
endBuffer = null;
|
||||
flattener.close();
|
||||
|
||||
try (CountingOutputStream metaOut = new CountingOutputStream(ioPeon.makeOutputStream(metaFile))) {
|
||||
metaOut.write(CompressedFloatsIndexedSupplier.version);
|
||||
metaOut.write(Ints.toByteArray(numInserted));
|
||||
metaOut.write(Ints.toByteArray(sizePer));
|
||||
metaOut.write(compression.getId());
|
||||
metaOut.close();
|
||||
metaCount = metaOut.getCount();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getSerializedSize()
|
||||
{
|
||||
return 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
if (endBuffer == null) {
|
||||
throw new IllegalStateException("written out already");
|
||||
}
|
||||
endBuffer = ByteBuffer.allocate(writer.getNumBytes(sizePer));
|
||||
writer.setBuffer(endBuffer);
|
||||
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.flip();
|
||||
if (endBuffer.remaining() > 0) {
|
||||
flattener.write(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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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
Loading…
Reference in New Issue