mirror of https://github.com/apache/druid.git
use mmap for nested column value to dictionary id lookup for more chill heap usage during serialization (#14919)
This commit is contained in:
parent
286eecad7c
commit
23b78c0f95
|
@ -228,13 +228,12 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
|
||||||
} else {
|
} else {
|
||||||
// all the bells and whistles
|
// all the bells and whistles
|
||||||
logicalType = ColumnType.NESTED_DATA;
|
logicalType = ColumnType.NESTED_DATA;
|
||||||
final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
|
serializer = new NestedDataColumnSerializer(
|
||||||
name,
|
name,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
);
|
);
|
||||||
serializer = defaultSerializer;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
serializer.openDictionaryWriter();
|
serializer.openDictionaryWriter();
|
||||||
|
|
|
@ -57,7 +57,7 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
|
||||||
private final Closer closer;
|
private final Closer closer;
|
||||||
|
|
||||||
private ColumnDescriptor.Builder descriptorBuilder;
|
private ColumnDescriptor.Builder descriptorBuilder;
|
||||||
private GenericColumnSerializer<?> serializer;
|
private NestedDataColumnSerializerV4 serializer;
|
||||||
|
|
||||||
public NestedDataColumnMergerV4(
|
public NestedDataColumnMergerV4(
|
||||||
String name,
|
String name,
|
||||||
|
@ -111,13 +111,12 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
|
||||||
|
|
||||||
descriptorBuilder = new ColumnDescriptor.Builder();
|
descriptorBuilder = new ColumnDescriptor.Builder();
|
||||||
|
|
||||||
final NestedDataColumnSerializerV4 defaultSerializer = new NestedDataColumnSerializerV4(
|
serializer = new NestedDataColumnSerializerV4(
|
||||||
name,
|
name,
|
||||||
indexSpec,
|
indexSpec,
|
||||||
segmentWriteOutMedium,
|
segmentWriteOutMedium,
|
||||||
closer
|
closer
|
||||||
);
|
);
|
||||||
serializer = defaultSerializer;
|
|
||||||
|
|
||||||
final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
|
final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
|
||||||
.withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
|
.withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
|
||||||
|
@ -127,14 +126,14 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
|
||||||
.setHasMultipleValues(false)
|
.setHasMultipleValues(false)
|
||||||
.addSerde(partSerde);
|
.addSerde(partSerde);
|
||||||
|
|
||||||
defaultSerializer.open();
|
serializer.open();
|
||||||
defaultSerializer.serializeFields(mergedFields);
|
serializer.serializeFields(mergedFields);
|
||||||
|
|
||||||
int stringCardinality;
|
int stringCardinality;
|
||||||
int longCardinality;
|
int longCardinality;
|
||||||
int doubleCardinality;
|
int doubleCardinality;
|
||||||
if (numMergeIndex == 1) {
|
if (numMergeIndex == 1) {
|
||||||
defaultSerializer.serializeDictionaries(
|
serializer.serializeDictionaries(
|
||||||
sortedLookup.getSortedStrings(),
|
sortedLookup.getSortedStrings(),
|
||||||
sortedLookup.getSortedLongs(),
|
sortedLookup.getSortedLongs(),
|
||||||
sortedLookup.getSortedDoubles()
|
sortedLookup.getSortedDoubles()
|
||||||
|
@ -155,7 +154,7 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
|
||||||
sortedDoubleLookups,
|
sortedDoubleLookups,
|
||||||
DOUBLE_MERGING_COMPARATOR
|
DOUBLE_MERGING_COMPARATOR
|
||||||
);
|
);
|
||||||
defaultSerializer.serializeDictionaries(
|
serializer.serializeDictionaries(
|
||||||
() -> stringIterator,
|
() -> stringIterator,
|
||||||
() -> longIterator,
|
() -> longIterator,
|
||||||
() -> doubleIterator
|
() -> doubleIterator
|
||||||
|
|
|
@ -19,12 +19,15 @@
|
||||||
|
|
||||||
package org.apache.druid.segment.column;
|
package org.apache.druid.segment.column;
|
||||||
|
|
||||||
|
import com.google.common.base.Supplier;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||||
import org.apache.druid.segment.IndexIO;
|
import org.apache.druid.segment.IndexIO;
|
||||||
import org.apache.druid.segment.data.DictionaryWriter;
|
import org.apache.druid.segment.data.DictionaryWriter;
|
||||||
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
|
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
|
||||||
|
import org.apache.druid.segment.data.FrontCodedIndexed;
|
||||||
import org.apache.druid.segment.data.FrontCodedIndexedWriter;
|
import org.apache.druid.segment.data.FrontCodedIndexedWriter;
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
import org.apache.druid.segment.data.GenericIndexedWriter;
|
import org.apache.druid.segment.data.GenericIndexedWriter;
|
||||||
|
@ -33,6 +36,7 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
|
||||||
public class StringEncodingStrategies
|
public class StringEncodingStrategies
|
||||||
|
@ -67,6 +71,39 @@ public class StringEncodingStrategies
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static Supplier<? extends Indexed<ByteBuffer>> getStringDictionarySupplier(
|
||||||
|
SmooshedFileMapper mapper,
|
||||||
|
ByteBuffer stringDictionaryBuffer,
|
||||||
|
ByteOrder byteOrder
|
||||||
|
)
|
||||||
|
{
|
||||||
|
final int dictionaryStartPosition = stringDictionaryBuffer.position();
|
||||||
|
final byte dictionaryVersion = stringDictionaryBuffer.get();
|
||||||
|
|
||||||
|
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
||||||
|
final byte encodingId = stringDictionaryBuffer.get();
|
||||||
|
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
||||||
|
return FrontCodedIndexed.read(
|
||||||
|
stringDictionaryBuffer,
|
||||||
|
byteOrder
|
||||||
|
);
|
||||||
|
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
||||||
|
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
||||||
|
// this provides backwards compatibility should we switch at some point in the future to always
|
||||||
|
// writing dictionaryVersion
|
||||||
|
return GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper)::singleThreaded;
|
||||||
|
} else {
|
||||||
|
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
// legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading
|
||||||
|
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
||||||
|
// GenericIndexed version can be correctly read
|
||||||
|
stringDictionaryBuffer.position(dictionaryStartPosition);
|
||||||
|
return GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper)::singleThreaded;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adapter to convert {@link Indexed<ByteBuffer>} with utf8 encoded bytes into {@link Indexed<String>} to be friendly
|
* Adapter to convert {@link Indexed<ByteBuffer>} with utf8 encoded bytes into {@link Indexed<String>} to be friendly
|
||||||
* to consumers.
|
* to consumers.
|
||||||
|
|
|
@ -155,7 +155,7 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
||||||
if (index == 0 && hasNulls) {
|
if (index == 0 && hasNulls) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
int startOffset = index * width;
|
int startOffset = (hasNulls ? index - 1 : index) * width;
|
||||||
readBuffer.clear();
|
readBuffer.clear();
|
||||||
valuesOut.readFully(startOffset, readBuffer);
|
valuesOut.readFully(startOffset, readBuffer);
|
||||||
readBuffer.clear();
|
readBuffer.clear();
|
||||||
|
@ -197,14 +197,14 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
|
||||||
{
|
{
|
||||||
iteratorBuffer.clear();
|
iteratorBuffer.clear();
|
||||||
try {
|
try {
|
||||||
if (totalCount - pos < PAGE_SIZE) {
|
if (numWritten - (pos - startPos) < PAGE_SIZE) {
|
||||||
int size = (totalCount - pos) * width;
|
int size = (numWritten - (pos - startPos)) * width;
|
||||||
iteratorBuffer.limit(size);
|
iteratorBuffer.limit(size);
|
||||||
valuesOut.readFully((long) pos * width, iteratorBuffer);
|
valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer);
|
||||||
} else {
|
} else {
|
||||||
valuesOut.readFully((long) pos * width, iteratorBuffer);
|
valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer);
|
||||||
}
|
}
|
||||||
iteratorBuffer.flip();
|
iteratorBuffer.clear();
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
throw new RuntimeException(e);
|
throw new RuntimeException(e);
|
||||||
|
|
|
@ -234,6 +234,7 @@ public class FrontCodedIndexedWriter implements DictionaryWriter<byte[]>
|
||||||
{
|
{
|
||||||
getOffsetBuffer.clear();
|
getOffsetBuffer.clear();
|
||||||
headerOut.readFully(index * (long) Integer.BYTES, getOffsetBuffer);
|
headerOut.readFully(index * (long) Integer.BYTES, getOffsetBuffer);
|
||||||
|
getOffsetBuffer.clear();
|
||||||
return getOffsetBuffer.getInt(0);
|
return getOffsetBuffer.getInt(0);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import it.unimi.dsi.fastutil.longs.LongArrayList;
|
import it.unimi.dsi.fastutil.longs.LongArrayList;
|
||||||
import it.unimi.dsi.fastutil.longs.LongList;
|
import it.unimi.dsi.fastutil.longs.LongList;
|
||||||
|
import org.apache.druid.common.config.NullHandling;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
|
@ -293,8 +294,17 @@ public class GenericIndexedWriter<T> implements DictionaryWriter<T>
|
||||||
long endOffset = getOffset(index);
|
long endOffset = getOffset(index);
|
||||||
int valueSize = checkedCastNonnegativeLongToInt(endOffset - startOffset);
|
int valueSize = checkedCastNonnegativeLongToInt(endOffset - startOffset);
|
||||||
if (valueSize == 0) {
|
if (valueSize == 0) {
|
||||||
|
if (NullHandling.replaceWithDefault()) {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
ByteBuffer bb = ByteBuffer.allocate(Integer.BYTES);
|
||||||
|
valuesOut.readFully(startOffset - Integer.BYTES, bb);
|
||||||
|
bb.flip();
|
||||||
|
if (bb.getInt() < 0) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
return strategy.fromByteBuffer(bb, 0);
|
||||||
|
}
|
||||||
ByteBuffer bb = ByteBuffer.allocate(valueSize);
|
ByteBuffer bb = ByteBuffer.allocate(valueSize);
|
||||||
valuesOut.readFully(startOffset, bb);
|
valuesOut.readFully(startOffset, bb);
|
||||||
bb.clear();
|
bb.clear();
|
||||||
|
|
|
@ -19,114 +19,308 @@
|
||||||
|
|
||||||
package org.apache.druid.segment.nested;
|
package org.apache.druid.segment.nested;
|
||||||
|
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.primitives.Ints;
|
||||||
import it.unimi.dsi.fastutil.doubles.Double2IntLinkedOpenHashMap;
|
import org.apache.druid.annotations.SuppressFBWarnings;
|
||||||
import it.unimi.dsi.fastutil.doubles.Double2IntMap;
|
import org.apache.druid.error.DruidException;
|
||||||
import it.unimi.dsi.fastutil.longs.Long2IntLinkedOpenHashMap;
|
import org.apache.druid.java.util.common.ByteBufferUtils;
|
||||||
import it.unimi.dsi.fastutil.longs.Long2IntMap;
|
import org.apache.druid.java.util.common.FileUtils;
|
||||||
import it.unimi.dsi.fastutil.objects.Object2IntAVLTreeMap;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import it.unimi.dsi.fastutil.objects.Object2IntLinkedOpenHashMap;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import it.unimi.dsi.fastutil.objects.Object2IntMap;
|
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter;
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter;
|
||||||
|
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||||
|
import org.apache.druid.segment.column.TypeStrategies;
|
||||||
|
import org.apache.druid.segment.data.DictionaryWriter;
|
||||||
|
import org.apache.druid.segment.data.FixedIndexed;
|
||||||
|
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
||||||
|
import org.apache.druid.segment.data.Indexed;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.nio.ByteBuffer;
|
||||||
|
import java.nio.ByteOrder;
|
||||||
|
import java.nio.MappedByteBuffer;
|
||||||
|
import java.nio.channels.FileChannel;
|
||||||
|
import java.nio.channels.GatheringByteChannel;
|
||||||
|
import java.nio.file.Files;
|
||||||
|
import java.nio.file.Path;
|
||||||
|
import java.nio.file.StandardOpenOption;
|
||||||
|
import java.util.EnumSet;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Ingestion time dictionary identifier lookup, used by {@link NestedCommonFormatColumnSerializer} to build a global
|
* Value to dictionary id lookup, backed with memory mapped dictionaries populated lazily by the supplied
|
||||||
* dictionary id to value mapping for the 'stacked' global value dictionaries.
|
* @link DictionaryWriter}.
|
||||||
*/
|
*/
|
||||||
public class DictionaryIdLookup
|
public final class DictionaryIdLookup implements Closeable
|
||||||
{
|
{
|
||||||
private final Object2IntMap<String> stringLookup;
|
private final String name;
|
||||||
|
@Nullable
|
||||||
|
private final DictionaryWriter<String> stringDictionaryWriter;
|
||||||
|
private SmooshedFileMapper stringBufferMapper = null;
|
||||||
|
private Indexed<ByteBuffer> stringDictionary = null;
|
||||||
|
|
||||||
private final Long2IntMap longLookup;
|
@Nullable
|
||||||
|
private final DictionaryWriter<Long> longDictionaryWriter;
|
||||||
|
private MappedByteBuffer longBuffer = null;
|
||||||
|
private FixedIndexed<Long> longDictionary = null;
|
||||||
|
|
||||||
private final Double2IntMap doubleLookup;
|
@Nullable
|
||||||
|
private final DictionaryWriter<Double> doubleDictionaryWriter;
|
||||||
|
MappedByteBuffer doubleBuffer = null;
|
||||||
|
FixedIndexed<Double> doubleDictionary = null;
|
||||||
|
|
||||||
private final Object2IntMap<int[]> arrayLookup;
|
@Nullable
|
||||||
|
private final DictionaryWriter<int[]> arrayDictionaryWriter;
|
||||||
|
private MappedByteBuffer arrayBuffer = null;
|
||||||
|
private FrontCodedIntArrayIndexed arrayDictionary = null;
|
||||||
|
|
||||||
private int dictionarySize;
|
public DictionaryIdLookup(
|
||||||
|
String name,
|
||||||
public DictionaryIdLookup()
|
@Nullable DictionaryWriter<String> stringDictionaryWriter,
|
||||||
|
@Nullable DictionaryWriter<Long> longDictionaryWriter,
|
||||||
|
@Nullable DictionaryWriter<Double> doubleDictionaryWriter,
|
||||||
|
@Nullable DictionaryWriter<int[]> arrayDictionaryWriter
|
||||||
|
)
|
||||||
{
|
{
|
||||||
this.stringLookup = new Object2IntLinkedOpenHashMap<>();
|
this.name = name;
|
||||||
stringLookup.defaultReturnValue(-1);
|
this.stringDictionaryWriter = stringDictionaryWriter;
|
||||||
this.longLookup = new Long2IntLinkedOpenHashMap();
|
this.longDictionaryWriter = longDictionaryWriter;
|
||||||
longLookup.defaultReturnValue(-1);
|
this.doubleDictionaryWriter = doubleDictionaryWriter;
|
||||||
this.doubleLookup = new Double2IntLinkedOpenHashMap();
|
this.arrayDictionaryWriter = arrayDictionaryWriter;
|
||||||
doubleLookup.defaultReturnValue(-1);
|
|
||||||
this.arrayLookup = new Object2IntAVLTreeMap<>(FrontCodedIntArrayIndexedWriter.ARRAY_COMPARATOR);
|
|
||||||
this.arrayLookup.defaultReturnValue(-1);
|
|
||||||
}
|
|
||||||
|
|
||||||
public void addString(@Nullable String value)
|
|
||||||
{
|
|
||||||
Preconditions.checkState(
|
|
||||||
longLookup.size() == 0 && doubleLookup.size() == 0,
|
|
||||||
"All string values must be inserted to the lookup before long and double types"
|
|
||||||
);
|
|
||||||
int id = dictionarySize++;
|
|
||||||
stringLookup.put(value, id);
|
|
||||||
}
|
|
||||||
|
|
||||||
// used when there are no string values to ensure that 0 is used for the null value
|
|
||||||
public void addNumericNull()
|
|
||||||
{
|
|
||||||
Preconditions.checkState(
|
|
||||||
stringLookup.size() == 0 && longLookup.size() == 0 && doubleLookup.size() == 0,
|
|
||||||
"Lookup must be empty to add implicit null"
|
|
||||||
);
|
|
||||||
dictionarySize++;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public int lookupString(@Nullable String value)
|
public int lookupString(@Nullable String value)
|
||||||
{
|
{
|
||||||
return stringLookup.getInt(value);
|
if (stringDictionary == null) {
|
||||||
}
|
// GenericIndexed v2 can write to multiple files if the dictionary is larger than 2gb, so we use a smooshfile
|
||||||
|
// for strings because of this. if other type dictionary writers could potentially use multiple internal files
|
||||||
public void addLong(long value)
|
// in the future, we should transition them to using this approach as well (or build a combination smoosher and
|
||||||
{
|
// mapper so that we can have a mutable smoosh)
|
||||||
Preconditions.checkState(
|
File stringSmoosh = FileUtils.createTempDir(name + "__stringTempSmoosh");
|
||||||
doubleLookup.size() == 0,
|
final String fileName = NestedCommonFormatColumnSerializer.getInternalFileName(
|
||||||
"All long values must be inserted to the lookup before double types"
|
name,
|
||||||
|
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
||||||
);
|
);
|
||||||
int id = dictionarySize++;
|
final FileSmoosher smoosher = new FileSmoosher(stringSmoosh);
|
||||||
longLookup.put(value, id);
|
try (final SmooshedWriter writer = smoosher.addWithSmooshedWriter(
|
||||||
|
fileName,
|
||||||
|
stringDictionaryWriter.getSerializedSize()
|
||||||
|
)) {
|
||||||
|
stringDictionaryWriter.writeTo(writer, smoosher);
|
||||||
|
writer.close();
|
||||||
|
smoosher.close();
|
||||||
|
stringBufferMapper = SmooshedFileMapper.load(stringSmoosh);
|
||||||
|
final ByteBuffer stringBuffer = stringBufferMapper.mapFile(fileName);
|
||||||
|
stringDictionary = StringEncodingStrategies.getStringDictionarySupplier(
|
||||||
|
stringBufferMapper,
|
||||||
|
stringBuffer,
|
||||||
|
ByteOrder.nativeOrder()
|
||||||
|
).get();
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
final byte[] bytes = StringUtils.toUtf8Nullable(value);
|
||||||
|
final int index = stringDictionary.indexOf(bytes == null ? null : ByteBuffer.wrap(bytes));
|
||||||
|
if (index < 0) {
|
||||||
|
throw DruidException.defensive("Value not found in string dictionary");
|
||||||
|
}
|
||||||
|
return index;
|
||||||
}
|
}
|
||||||
|
|
||||||
public int lookupLong(@Nullable Long value)
|
public int lookupLong(@Nullable Long value)
|
||||||
{
|
{
|
||||||
if (value == null) {
|
if (longDictionary == null) {
|
||||||
return 0;
|
Path longFile = makeTempFile(name + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME);
|
||||||
|
longBuffer = mapWriter(longFile, longDictionaryWriter);
|
||||||
|
longDictionary = FixedIndexed.read(longBuffer, TypeStrategies.LONG, ByteOrder.nativeOrder(), Long.BYTES).get();
|
||||||
|
// reset position
|
||||||
|
longBuffer.position(0);
|
||||||
}
|
}
|
||||||
return longLookup.get(value.longValue());
|
final int index = longDictionary.indexOf(value);
|
||||||
|
if (index < 0) {
|
||||||
|
throw DruidException.defensive("Value not found in long dictionary");
|
||||||
}
|
}
|
||||||
|
return index + longOffset();
|
||||||
public void addDouble(double value)
|
|
||||||
{
|
|
||||||
int id = dictionarySize++;
|
|
||||||
doubleLookup.put(value, id);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public int lookupDouble(@Nullable Double value)
|
public int lookupDouble(@Nullable Double value)
|
||||||
{
|
{
|
||||||
if (value == null) {
|
if (doubleDictionary == null) {
|
||||||
return 0;
|
Path doubleFile = makeTempFile(name + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME);
|
||||||
|
doubleBuffer = mapWriter(doubleFile, doubleDictionaryWriter);
|
||||||
|
doubleDictionary = FixedIndexed.read(doubleBuffer, TypeStrategies.DOUBLE, ByteOrder.nativeOrder(), Double.BYTES).get();
|
||||||
|
// reset position
|
||||||
|
doubleBuffer.position(0);
|
||||||
}
|
}
|
||||||
return doubleLookup.get(value.doubleValue());
|
final int index = doubleDictionary.indexOf(value);
|
||||||
|
if (index < 0) {
|
||||||
|
throw DruidException.defensive("Value not found in double dictionary");
|
||||||
}
|
}
|
||||||
|
return index + doubleOffset();
|
||||||
public void addArray(int[] value)
|
|
||||||
{
|
|
||||||
int id = dictionarySize++;
|
|
||||||
arrayLookup.put(value, id);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public int lookupArray(@Nullable int[] value)
|
public int lookupArray(@Nullable int[] value)
|
||||||
{
|
{
|
||||||
if (value == null) {
|
if (arrayDictionary == null) {
|
||||||
return 0;
|
Path arrayFile = makeTempFile(name + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME);
|
||||||
|
arrayBuffer = mapWriter(arrayFile, arrayDictionaryWriter);
|
||||||
|
arrayDictionary = FrontCodedIntArrayIndexed.read(arrayBuffer, ByteOrder.nativeOrder()).get();
|
||||||
|
// reset position
|
||||||
|
arrayBuffer.position(0);
|
||||||
}
|
}
|
||||||
return arrayLookup.getInt(value);
|
final int index = arrayDictionary.indexOf(value);
|
||||||
|
if (index < 0) {
|
||||||
|
throw DruidException.defensive("Value not found in array dictionary");
|
||||||
|
}
|
||||||
|
return index + arrayOffset();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public SmooshedFileMapper getStringBufferMapper()
|
||||||
|
{
|
||||||
|
return stringBufferMapper;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public ByteBuffer getLongBuffer()
|
||||||
|
{
|
||||||
|
return longBuffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public ByteBuffer getDoubleBuffer()
|
||||||
|
{
|
||||||
|
return doubleBuffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
public ByteBuffer getArrayBuffer()
|
||||||
|
{
|
||||||
|
return arrayBuffer;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close()
|
||||||
|
{
|
||||||
|
if (stringBufferMapper != null) {
|
||||||
|
stringBufferMapper.close();
|
||||||
|
}
|
||||||
|
if (longBuffer != null) {
|
||||||
|
ByteBufferUtils.unmap(longBuffer);
|
||||||
|
}
|
||||||
|
if (doubleBuffer != null) {
|
||||||
|
ByteBufferUtils.unmap(doubleBuffer);
|
||||||
|
}
|
||||||
|
if (arrayBuffer != null) {
|
||||||
|
ByteBufferUtils.unmap(arrayBuffer);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private int longOffset()
|
||||||
|
{
|
||||||
|
return stringDictionaryWriter != null ? stringDictionaryWriter.getCardinality() : 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
private int doubleOffset()
|
||||||
|
{
|
||||||
|
return longOffset() + (longDictionaryWriter != null ? longDictionaryWriter.getCardinality() : 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
private int arrayOffset()
|
||||||
|
{
|
||||||
|
return doubleOffset() + (doubleDictionaryWriter != null ? doubleDictionaryWriter.getCardinality() : 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
private Path makeTempFile(String name)
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
return Files.createTempFile(name, ".tmp");
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
|
||||||
|
private MappedByteBuffer mapWriter(Path path, DictionaryWriter<?> writer)
|
||||||
|
{
|
||||||
|
final EnumSet<StandardOpenOption> options = EnumSet.of(
|
||||||
|
StandardOpenOption.READ,
|
||||||
|
StandardOpenOption.WRITE,
|
||||||
|
StandardOpenOption.CREATE,
|
||||||
|
StandardOpenOption.TRUNCATE_EXISTING
|
||||||
|
);
|
||||||
|
|
||||||
|
try (FileChannel fileChannel = FileChannel.open(path, options);
|
||||||
|
GatheringByteChannel smooshChannel = makeWriter(fileChannel, writer.getSerializedSize())) {
|
||||||
|
//noinspection DataFlowIssue
|
||||||
|
writer.writeTo(smooshChannel, null);
|
||||||
|
return fileChannel.map(FileChannel.MapMode.READ_ONLY, 0, writer.getSerializedSize());
|
||||||
|
}
|
||||||
|
catch (IOException e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private GatheringByteChannel makeWriter(FileChannel channel, long size)
|
||||||
|
{
|
||||||
|
// basically same code as smooshed writer, can't use channel directly because copying between channels
|
||||||
|
// doesn't handle size of source channel correctly
|
||||||
|
return new GatheringByteChannel()
|
||||||
|
{
|
||||||
|
private boolean isClosed = false;
|
||||||
|
private long currOffset = 0;
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean isOpen()
|
||||||
|
{
|
||||||
|
return !isClosed;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
channel.close();
|
||||||
|
isClosed = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
public int bytesLeft()
|
||||||
|
{
|
||||||
|
return (int) (size - currOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int write(ByteBuffer buffer) throws IOException
|
||||||
|
{
|
||||||
|
return addToOffset(channel.write(buffer));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long write(ByteBuffer[] srcs, int offset, int length) throws IOException
|
||||||
|
{
|
||||||
|
return addToOffset(channel.write(srcs, offset, length));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public long write(ByteBuffer[] srcs) throws IOException
|
||||||
|
{
|
||||||
|
return addToOffset(channel.write(srcs));
|
||||||
|
}
|
||||||
|
|
||||||
|
public int addToOffset(long numBytesWritten)
|
||||||
|
{
|
||||||
|
if (numBytesWritten > bytesLeft()) {
|
||||||
|
throw new ISE("Wrote more bytes[%,d] than available[%,d]. Don't do that.", numBytesWritten, bytesLeft());
|
||||||
|
}
|
||||||
|
currOffset += numBytesWritten;
|
||||||
|
|
||||||
|
return Ints.checkedCast(numBytesWritten);
|
||||||
|
}
|
||||||
|
};
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -92,6 +92,14 @@ public abstract class NestedCommonFormatColumnSerializer implements GenericColum
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected void writeInternal(FileSmoosher smoosher, ByteBuffer buffer, String fileName) throws IOException
|
||||||
|
{
|
||||||
|
final String internalName = getInternalFileName(getColumnName(), fileName);
|
||||||
|
try (SmooshedWriter smooshChannel = smoosher.addWithSmooshedWriter(internalName, buffer.capacity())) {
|
||||||
|
smooshChannel.write(buffer);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
protected void writeV0Header(WritableByteChannel channel, ByteBuffer columnNameBuffer) throws IOException
|
protected void writeV0Header(WritableByteChannel channel, ByteBuffer columnNameBuffer) throws IOException
|
||||||
{
|
{
|
||||||
channel.write(ByteBuffer.wrap(new byte[]{NestedCommonFormatColumnSerializer.V0}));
|
channel.write(ByteBuffer.wrap(new byte[]{NestedCommonFormatColumnSerializer.V0}));
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.RE;
|
import org.apache.druid.java.util.common.RE;
|
||||||
import org.apache.druid.java.util.common.io.Closer;
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||||
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter;
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
|
@ -170,7 +171,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||||
this.indexSpec = indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
this.globalDictionaryIdLookup = new DictionaryIdLookup();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -231,6 +231,15 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
4
|
4
|
||||||
);
|
);
|
||||||
arrayDictionaryWriter.open();
|
arrayDictionaryWriter.open();
|
||||||
|
globalDictionaryIdLookup = closer.register(
|
||||||
|
new DictionaryIdLookup(
|
||||||
|
name,
|
||||||
|
dictionaryWriter,
|
||||||
|
longDictionaryWriter,
|
||||||
|
doubleDictionaryWriter,
|
||||||
|
arrayDictionaryWriter
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -329,7 +338,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
|
|
||||||
// null is always 0
|
// null is always 0
|
||||||
dictionaryWriter.write(null);
|
dictionaryWriter.write(null);
|
||||||
globalDictionaryIdLookup.addString(null);
|
|
||||||
for (String value : strings) {
|
for (String value : strings) {
|
||||||
value = NullHandling.emptyToNullIfNeeded(value);
|
value = NullHandling.emptyToNullIfNeeded(value);
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
|
@ -337,7 +345,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
}
|
}
|
||||||
|
|
||||||
dictionaryWriter.write(value);
|
dictionaryWriter.write(value);
|
||||||
globalDictionaryIdLookup.addString(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
|
|
||||||
|
@ -346,7 +353,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
longDictionaryWriter.write(value);
|
longDictionaryWriter.write(value);
|
||||||
globalDictionaryIdLookup.addLong(value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (Double value : doubles) {
|
for (Double value : doubles) {
|
||||||
|
@ -354,7 +360,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
doubleDictionaryWriter.write(value);
|
doubleDictionaryWriter.write(value);
|
||||||
globalDictionaryIdLookup.addDouble(value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (int[] value : arrays) {
|
for (int[] value : arrays) {
|
||||||
|
@ -362,7 +367,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
arrayDictionaryWriter.write(value);
|
arrayDictionaryWriter.write(value);
|
||||||
globalDictionaryIdLookup.addArray(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
}
|
}
|
||||||
|
@ -433,10 +437,31 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
|
||||||
writeV0Header(channel, columnNameBytes);
|
writeV0Header(channel, columnNameBytes);
|
||||||
fieldsWriter.writeTo(channel, smoosher);
|
fieldsWriter.writeTo(channel, smoosher);
|
||||||
fieldsInfoWriter.writeTo(channel, smoosher);
|
fieldsInfoWriter.writeTo(channel, smoosher);
|
||||||
|
|
||||||
|
|
||||||
|
if (globalDictionaryIdLookup.getStringBufferMapper() != null) {
|
||||||
|
SmooshedFileMapper fileMapper = globalDictionaryIdLookup.getStringBufferMapper();
|
||||||
|
for (String internalName : fileMapper.getInternalFilenames()) {
|
||||||
|
smoosher.add(internalName, fileMapper.mapFile(internalName));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
if (globalDictionaryIdLookup.getLongBuffer() != null) {
|
||||||
|
writeInternal(smoosher, globalDictionaryIdLookup.getLongBuffer(), LONG_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
if (globalDictionaryIdLookup.getDoubleBuffer() != null) {
|
||||||
|
writeInternal(smoosher, globalDictionaryIdLookup.getDoubleBuffer(), DOUBLE_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
if (globalDictionaryIdLookup.getArrayBuffer() != null) {
|
||||||
|
writeInternal(smoosher, globalDictionaryIdLookup.getArrayBuffer(), ARRAY_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
writeInternal(smoosher, rawWriter, RAW_FILE_NAME);
|
writeInternal(smoosher, rawWriter, RAW_FILE_NAME);
|
||||||
if (!nullRowsBitmap.isEmpty()) {
|
if (!nullRowsBitmap.isEmpty()) {
|
||||||
writeInternal(smoosher, nullBitmapWriter, NULL_BITMAP_FILE_NAME);
|
writeInternal(smoosher, nullBitmapWriter, NULL_BITMAP_FILE_NAME);
|
||||||
|
|
|
@ -144,7 +144,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
|
||||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||||
this.indexSpec = indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
this.globalDictionaryIdLookup = new DictionaryIdLookup();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -195,6 +194,16 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
|
||||||
nullBitmapWriter.open();
|
nullBitmapWriter.open();
|
||||||
|
|
||||||
nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
|
nullRowsBitmap = indexSpec.getBitmapSerdeFactory().getBitmapFactory().makeEmptyMutableBitmap();
|
||||||
|
|
||||||
|
globalDictionaryIdLookup = closer.register(
|
||||||
|
new DictionaryIdLookup(
|
||||||
|
name,
|
||||||
|
dictionaryWriter,
|
||||||
|
longDictionaryWriter,
|
||||||
|
doubleDictionaryWriter,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
public void serializeFields(SortedMap<String, FieldTypeInfo.MutableTypeSet> fields) throws IOException
|
public void serializeFields(SortedMap<String, FieldTypeInfo.MutableTypeSet> fields) throws IOException
|
||||||
|
@ -263,7 +272,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
|
||||||
|
|
||||||
// null is always 0
|
// null is always 0
|
||||||
dictionaryWriter.write(null);
|
dictionaryWriter.write(null);
|
||||||
globalDictionaryIdLookup.addString(null);
|
|
||||||
for (String value : strings) {
|
for (String value : strings) {
|
||||||
value = NullHandling.emptyToNullIfNeeded(value);
|
value = NullHandling.emptyToNullIfNeeded(value);
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
|
@ -271,7 +279,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
|
||||||
}
|
}
|
||||||
|
|
||||||
dictionaryWriter.write(value);
|
dictionaryWriter.write(value);
|
||||||
globalDictionaryIdLookup.addString(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
|
|
||||||
|
@ -280,7 +287,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
longDictionaryWriter.write(value);
|
longDictionaryWriter.write(value);
|
||||||
globalDictionaryIdLookup.addLong(value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (Double value : doubles) {
|
for (Double value : doubles) {
|
||||||
|
@ -288,7 +294,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
doubleDictionaryWriter.write(value);
|
doubleDictionaryWriter.write(value);
|
||||||
globalDictionaryIdLookup.addDouble(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
}
|
}
|
||||||
|
|
|
@ -21,21 +21,19 @@ package org.apache.druid.segment.nested;
|
||||||
|
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
|
||||||
import org.apache.druid.java.util.common.RE;
|
import org.apache.druid.java.util.common.RE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.segment.column.ColumnBuilder;
|
import org.apache.druid.segment.column.ColumnBuilder;
|
||||||
import org.apache.druid.segment.column.ColumnConfig;
|
import org.apache.druid.segment.column.ColumnConfig;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||||
import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier;
|
import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier;
|
||||||
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
|
|
||||||
import org.apache.druid.segment.data.FixedIndexed;
|
import org.apache.druid.segment.data.FixedIndexed;
|
||||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
|
||||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
|
import org.apache.druid.segment.data.Indexed;
|
||||||
import org.apache.druid.segment.data.VByte;
|
import org.apache.druid.segment.data.VByte;
|
||||||
import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
|
import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
|
||||||
|
|
||||||
|
@ -67,8 +65,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
final FieldTypeInfo fieldInfo;
|
final FieldTypeInfo fieldInfo;
|
||||||
final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
||||||
final ImmutableBitmap nullValues;
|
final ImmutableBitmap nullValues;
|
||||||
final GenericIndexed<ByteBuffer> stringDictionary;
|
final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
|
||||||
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
|
||||||
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
||||||
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||||
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
||||||
|
@ -82,34 +79,12 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
||||||
);
|
);
|
||||||
|
|
||||||
final int dictionaryStartPosition = stringDictionaryBuffer.position();
|
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
|
||||||
final byte dictionaryVersion = stringDictionaryBuffer.get();
|
mapper,
|
||||||
|
|
||||||
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
|
||||||
final byte encodingId = stringDictionaryBuffer.get();
|
|
||||||
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
|
||||||
frontCodedStringDictionarySupplier = FrontCodedIndexed.read(
|
|
||||||
stringDictionaryBuffer,
|
stringDictionaryBuffer,
|
||||||
byteOrder
|
byteOrder
|
||||||
);
|
);
|
||||||
stringDictionary = null;
|
|
||||||
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
|
||||||
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
|
||||||
// this provides backwards compatibility should we switch at some point in the future to always
|
|
||||||
// writing dictionaryVersion
|
|
||||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
|
||||||
frontCodedStringDictionarySupplier = null;
|
|
||||||
} else {
|
|
||||||
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading
|
|
||||||
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
|
||||||
// GenericIndexed version can be correctly read
|
|
||||||
stringDictionaryBuffer.position(dictionaryStartPosition);
|
|
||||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
|
||||||
frontCodedStringDictionarySupplier = null;
|
|
||||||
}
|
|
||||||
final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
final ByteBuffer longDictionaryBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
||||||
mapper,
|
mapper,
|
||||||
columnName,
|
columnName,
|
||||||
|
@ -173,8 +148,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
fieldInfo,
|
fieldInfo,
|
||||||
compressedRawColumnSupplier,
|
compressedRawColumnSupplier,
|
||||||
nullValues,
|
nullValues,
|
||||||
stringDictionary,
|
stringDictionarySupplier,
|
||||||
frontCodedStringDictionarySupplier,
|
|
||||||
longDictionarySupplier,
|
longDictionarySupplier,
|
||||||
doubleDictionarySupplier,
|
doubleDictionarySupplier,
|
||||||
arrayDictionarySupplier,
|
arrayDictionarySupplier,
|
||||||
|
@ -198,8 +172,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
private final FieldTypeInfo fieldInfo;
|
private final FieldTypeInfo fieldInfo;
|
||||||
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
||||||
private final ImmutableBitmap nullValues;
|
private final ImmutableBitmap nullValues;
|
||||||
private final GenericIndexed<ByteBuffer> stringDictionary;
|
private final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
|
||||||
private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
|
||||||
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
||||||
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||||
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
||||||
|
@ -217,8 +190,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
FieldTypeInfo fieldInfo,
|
FieldTypeInfo fieldInfo,
|
||||||
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
|
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
|
||||||
ImmutableBitmap nullValues,
|
ImmutableBitmap nullValues,
|
||||||
GenericIndexed<ByteBuffer> stringDictionary,
|
Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier,
|
||||||
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
|
|
||||||
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
||||||
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
||||||
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
||||||
|
@ -234,8 +206,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
this.fieldInfo = fieldInfo;
|
this.fieldInfo = fieldInfo;
|
||||||
this.compressedRawColumnSupplier = compressedRawColumnSupplier;
|
this.compressedRawColumnSupplier = compressedRawColumnSupplier;
|
||||||
this.nullValues = nullValues;
|
this.nullValues = nullValues;
|
||||||
this.stringDictionary = stringDictionary;
|
this.stringDictionarySupplier = stringDictionarySupplier;
|
||||||
this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier;
|
|
||||||
this.longDictionarySupplier = longDictionarySupplier;
|
this.longDictionarySupplier = longDictionarySupplier;
|
||||||
this.doubleDictionarySupplier = doubleDictionarySupplier;
|
this.doubleDictionarySupplier = doubleDictionarySupplier;
|
||||||
this.arrayDictionarySupplier = arrayDictionarySupplier;
|
this.arrayDictionarySupplier = arrayDictionarySupplier;
|
||||||
|
@ -249,7 +220,6 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
@Override
|
@Override
|
||||||
public NestedCommonFormatColumn get()
|
public NestedCommonFormatColumn get()
|
||||||
{
|
{
|
||||||
if (frontCodedStringDictionarySupplier != null) {
|
|
||||||
return new NestedDataColumnV5<>(
|
return new NestedDataColumnV5<>(
|
||||||
columnName,
|
columnName,
|
||||||
getLogicalType(),
|
getLogicalType(),
|
||||||
|
@ -258,24 +228,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
|
||||||
nullValues,
|
nullValues,
|
||||||
fields,
|
fields,
|
||||||
fieldInfo,
|
fieldInfo,
|
||||||
frontCodedStringDictionarySupplier,
|
stringDictionarySupplier,
|
||||||
longDictionarySupplier,
|
|
||||||
doubleDictionarySupplier,
|
|
||||||
arrayDictionarySupplier,
|
|
||||||
fileMapper,
|
|
||||||
bitmapSerdeFactory,
|
|
||||||
byteOrder
|
|
||||||
);
|
|
||||||
}
|
|
||||||
return new NestedDataColumnV5<>(
|
|
||||||
columnName,
|
|
||||||
getLogicalType(),
|
|
||||||
columnConfig,
|
|
||||||
compressedRawColumnSupplier,
|
|
||||||
nullValues,
|
|
||||||
fields,
|
|
||||||
fieldInfo,
|
|
||||||
stringDictionary::singleThreaded,
|
|
||||||
longDictionarySupplier,
|
longDictionarySupplier,
|
||||||
doubleDictionarySupplier,
|
doubleDictionarySupplier,
|
||||||
arrayDictionarySupplier,
|
arrayDictionarySupplier,
|
||||||
|
|
|
@ -22,7 +22,6 @@ package org.apache.druid.segment.nested;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
|
||||||
import org.apache.druid.java.util.common.RE;
|
import org.apache.druid.java.util.common.RE;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.segment.IndexMerger;
|
import org.apache.druid.segment.IndexMerger;
|
||||||
|
@ -30,15 +29,14 @@ import org.apache.druid.segment.column.ColumnBuilder;
|
||||||
import org.apache.druid.segment.column.ColumnConfig;
|
import org.apache.druid.segment.column.ColumnConfig;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.ComplexColumn;
|
import org.apache.druid.segment.column.ComplexColumn;
|
||||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||||
import org.apache.druid.segment.column.TypeStrategy;
|
import org.apache.druid.segment.column.TypeStrategy;
|
||||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||||
import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier;
|
import org.apache.druid.segment.data.CompressedVariableSizedBlobColumnSupplier;
|
||||||
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
|
|
||||||
import org.apache.druid.segment.data.FixedIndexed;
|
import org.apache.druid.segment.data.FixedIndexed;
|
||||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
|
||||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
|
import org.apache.druid.segment.data.Indexed;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -84,8 +82,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
final FieldTypeInfo fieldInfo;
|
final FieldTypeInfo fieldInfo;
|
||||||
final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
||||||
final ImmutableBitmap nullValues;
|
final ImmutableBitmap nullValues;
|
||||||
final GenericIndexed<ByteBuffer> stringDictionary;
|
final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
|
||||||
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
|
||||||
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
||||||
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||||
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
||||||
|
@ -118,34 +115,11 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
||||||
);
|
);
|
||||||
|
|
||||||
final int dictionaryStartPosition = stringDictionaryBuffer.position();
|
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
|
||||||
final byte dictionaryVersion = stringDictionaryBuffer.get();
|
mapper,
|
||||||
|
|
||||||
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
|
||||||
final byte encodingId = stringDictionaryBuffer.get();
|
|
||||||
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
|
||||||
frontCodedStringDictionarySupplier = FrontCodedIndexed.read(
|
|
||||||
stringDictionaryBuffer,
|
stringDictionaryBuffer,
|
||||||
metadata.getByteOrder()
|
metadata.getByteOrder()
|
||||||
);
|
);
|
||||||
stringDictionary = null;
|
|
||||||
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
|
||||||
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
|
||||||
// this provides backwards compatibility should we switch at some point in the future to always
|
|
||||||
// writing dictionaryVersion
|
|
||||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
|
||||||
frontCodedStringDictionarySupplier = null;
|
|
||||||
} else {
|
|
||||||
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading
|
|
||||||
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
|
||||||
// GenericIndexed version can be correctly read
|
|
||||||
stringDictionaryBuffer.position(dictionaryStartPosition);
|
|
||||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
|
||||||
frontCodedStringDictionarySupplier = null;
|
|
||||||
}
|
|
||||||
final ByteBuffer longDictionaryBuffer = loadInternalFile(
|
final ByteBuffer longDictionaryBuffer = loadInternalFile(
|
||||||
mapper,
|
mapper,
|
||||||
metadata,
|
metadata,
|
||||||
|
@ -210,8 +184,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
fieldInfo,
|
fieldInfo,
|
||||||
compressedRawColumnSupplier,
|
compressedRawColumnSupplier,
|
||||||
nullValues,
|
nullValues,
|
||||||
stringDictionary,
|
stringDictionarySupplier,
|
||||||
frontCodedStringDictionarySupplier,
|
|
||||||
longDictionarySupplier,
|
longDictionarySupplier,
|
||||||
doubleDictionarySupplier,
|
doubleDictionarySupplier,
|
||||||
arrayDictionarySupplier,
|
arrayDictionarySupplier,
|
||||||
|
@ -236,8 +209,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
private final FieldTypeInfo fieldInfo;
|
private final FieldTypeInfo fieldInfo;
|
||||||
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
|
||||||
private final ImmutableBitmap nullValues;
|
private final ImmutableBitmap nullValues;
|
||||||
private final GenericIndexed<ByteBuffer> stringDictionary;
|
private final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
|
||||||
private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
|
||||||
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
||||||
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||||
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
||||||
|
@ -257,8 +229,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
FieldTypeInfo fieldInfo,
|
FieldTypeInfo fieldInfo,
|
||||||
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
|
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
|
||||||
ImmutableBitmap nullValues,
|
ImmutableBitmap nullValues,
|
||||||
GenericIndexed<ByteBuffer> stringDictionary,
|
Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier,
|
||||||
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
|
|
||||||
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
||||||
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
||||||
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
||||||
|
@ -275,8 +246,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
this.fieldInfo = fieldInfo;
|
this.fieldInfo = fieldInfo;
|
||||||
this.compressedRawColumnSupplier = compressedRawColumnSupplier;
|
this.compressedRawColumnSupplier = compressedRawColumnSupplier;
|
||||||
this.nullValues = nullValues;
|
this.nullValues = nullValues;
|
||||||
this.stringDictionary = stringDictionary;
|
this.stringDictionarySupplier = stringDictionarySupplier;
|
||||||
this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier;
|
|
||||||
this.longDictionarySupplier = longDictionarySupplier;
|
this.longDictionarySupplier = longDictionarySupplier;
|
||||||
this.doubleDictionarySupplier = doubleDictionarySupplier;
|
this.doubleDictionarySupplier = doubleDictionarySupplier;
|
||||||
this.arrayDictionarySupplier = arrayDictionarySupplier;
|
this.arrayDictionarySupplier = arrayDictionarySupplier;
|
||||||
|
@ -307,7 +277,6 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
|
|
||||||
private NestedDataColumnV3 makeV3()
|
private NestedDataColumnV3 makeV3()
|
||||||
{
|
{
|
||||||
if (frontCodedStringDictionarySupplier != null) {
|
|
||||||
return new NestedDataColumnV3<>(
|
return new NestedDataColumnV3<>(
|
||||||
columnName,
|
columnName,
|
||||||
logicalType,
|
logicalType,
|
||||||
|
@ -316,23 +285,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
nullValues,
|
nullValues,
|
||||||
fields,
|
fields,
|
||||||
fieldInfo,
|
fieldInfo,
|
||||||
frontCodedStringDictionarySupplier,
|
stringDictionarySupplier,
|
||||||
longDictionarySupplier,
|
|
||||||
doubleDictionarySupplier,
|
|
||||||
fileMapper,
|
|
||||||
bitmapSerdeFactory,
|
|
||||||
byteOrder
|
|
||||||
);
|
|
||||||
}
|
|
||||||
return new NestedDataColumnV3<>(
|
|
||||||
columnName,
|
|
||||||
logicalType,
|
|
||||||
columnConfig,
|
|
||||||
compressedRawColumnSupplier,
|
|
||||||
nullValues,
|
|
||||||
fields,
|
|
||||||
fieldInfo,
|
|
||||||
stringDictionary::singleThreaded,
|
|
||||||
longDictionarySupplier,
|
longDictionarySupplier,
|
||||||
doubleDictionarySupplier,
|
doubleDictionarySupplier,
|
||||||
fileMapper,
|
fileMapper,
|
||||||
|
@ -343,7 +296,6 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
|
|
||||||
private NestedDataColumnV4 makeV4()
|
private NestedDataColumnV4 makeV4()
|
||||||
{
|
{
|
||||||
if (frontCodedStringDictionarySupplier != null) {
|
|
||||||
return new NestedDataColumnV4<>(
|
return new NestedDataColumnV4<>(
|
||||||
columnName,
|
columnName,
|
||||||
logicalType,
|
logicalType,
|
||||||
|
@ -352,23 +304,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
nullValues,
|
nullValues,
|
||||||
fields,
|
fields,
|
||||||
fieldInfo,
|
fieldInfo,
|
||||||
frontCodedStringDictionarySupplier,
|
stringDictionarySupplier,
|
||||||
longDictionarySupplier,
|
|
||||||
doubleDictionarySupplier,
|
|
||||||
fileMapper,
|
|
||||||
bitmapSerdeFactory,
|
|
||||||
byteOrder
|
|
||||||
);
|
|
||||||
}
|
|
||||||
return new NestedDataColumnV4<>(
|
|
||||||
columnName,
|
|
||||||
logicalType,
|
|
||||||
columnConfig,
|
|
||||||
compressedRawColumnSupplier,
|
|
||||||
nullValues,
|
|
||||||
fields,
|
|
||||||
fieldInfo,
|
|
||||||
stringDictionary::singleThreaded,
|
|
||||||
longDictionarySupplier,
|
longDictionarySupplier,
|
||||||
doubleDictionarySupplier,
|
doubleDictionarySupplier,
|
||||||
fileMapper,
|
fileMapper,
|
||||||
|
@ -379,7 +315,6 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
|
|
||||||
private NestedDataColumnV5 makeV5()
|
private NestedDataColumnV5 makeV5()
|
||||||
{
|
{
|
||||||
if (frontCodedStringDictionarySupplier != null) {
|
|
||||||
return new NestedDataColumnV5<>(
|
return new NestedDataColumnV5<>(
|
||||||
columnName,
|
columnName,
|
||||||
logicalType,
|
logicalType,
|
||||||
|
@ -388,24 +323,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
|
||||||
nullValues,
|
nullValues,
|
||||||
fields,
|
fields,
|
||||||
fieldInfo,
|
fieldInfo,
|
||||||
frontCodedStringDictionarySupplier,
|
stringDictionarySupplier,
|
||||||
longDictionarySupplier,
|
|
||||||
doubleDictionarySupplier,
|
|
||||||
arrayDictionarySupplier,
|
|
||||||
fileMapper,
|
|
||||||
bitmapSerdeFactory,
|
|
||||||
byteOrder
|
|
||||||
);
|
|
||||||
}
|
|
||||||
return new NestedDataColumnV5<>(
|
|
||||||
columnName,
|
|
||||||
logicalType,
|
|
||||||
columnConfig,
|
|
||||||
compressedRawColumnSupplier,
|
|
||||||
nullValues,
|
|
||||||
fields,
|
|
||||||
fieldInfo,
|
|
||||||
stringDictionary::singleThreaded,
|
|
||||||
longDictionarySupplier,
|
longDictionarySupplier,
|
||||||
doubleDictionarySupplier,
|
doubleDictionarySupplier,
|
||||||
arrayDictionarySupplier,
|
arrayDictionarySupplier,
|
||||||
|
|
|
@ -49,7 +49,7 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
Closer closer
|
Closer closer
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(name, DOUBLE_DICTIONARY_FILE_NAME, indexSpec, segmentWriteOutMedium, closer);
|
super(name, indexSpec, segmentWriteOutMedium, closer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -73,6 +73,15 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
dictionaryWriter.open();
|
dictionaryWriter.open();
|
||||||
|
dictionaryIdLookup = closer.register(
|
||||||
|
new DictionaryIdLookup(
|
||||||
|
name,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
dictionaryWriter,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -102,16 +111,15 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
|
|
||||||
// null is always 0
|
// null is always 0
|
||||||
dictionaryWriter.write(null);
|
dictionaryWriter.write(null);
|
||||||
dictionaryIdLookup.addNumericNull();
|
|
||||||
|
|
||||||
for (Double value : doubles) {
|
for (Double value : doubles) {
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
dictionaryWriter.write(value);
|
dictionaryWriter.write(value);
|
||||||
dictionaryIdLookup.addDouble(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -119,4 +127,14 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
{
|
{
|
||||||
writeInternal(smoosher, doublesSerializer, DOUBLE_VALUE_COLUMN_FILE_NAME);
|
writeInternal(smoosher, doublesSerializer, DOUBLE_VALUE_COLUMN_FILE_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void writeDictionaryFile(FileSmoosher smoosher) throws IOException
|
||||||
|
{
|
||||||
|
if (dictionaryIdLookup.getDoubleBuffer() != null) {
|
||||||
|
writeInternal(smoosher, dictionaryIdLookup.getDoubleBuffer(), DOUBLE_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
|
writeInternal(smoosher, dictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -49,7 +49,7 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
|
||||||
Closer closer
|
Closer closer
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(name, LONG_DICTIONARY_FILE_NAME, indexSpec, segmentWriteOutMedium, closer);
|
super(name, indexSpec, segmentWriteOutMedium, closer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -74,6 +74,15 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
|
||||||
true
|
true
|
||||||
);
|
);
|
||||||
dictionaryWriter.open();
|
dictionaryWriter.open();
|
||||||
|
dictionaryIdLookup = closer.register(
|
||||||
|
new DictionaryIdLookup(
|
||||||
|
name,
|
||||||
|
null,
|
||||||
|
dictionaryWriter,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -104,14 +113,11 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
|
||||||
|
|
||||||
// null is always 0
|
// null is always 0
|
||||||
dictionaryWriter.write(null);
|
dictionaryWriter.write(null);
|
||||||
dictionaryIdLookup.addNumericNull();
|
|
||||||
|
|
||||||
for (Long value : longs) {
|
for (Long value : longs) {
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
dictionaryWriter.write(value);
|
dictionaryWriter.write(value);
|
||||||
dictionaryIdLookup.addLong(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
}
|
}
|
||||||
|
@ -121,4 +127,14 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
|
||||||
{
|
{
|
||||||
writeInternal(smoosher, longsSerializer, LONG_VALUE_COLUMN_FILE_NAME);
|
writeInternal(smoosher, longsSerializer, LONG_VALUE_COLUMN_FILE_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void writeDictionaryFile(FileSmoosher smoosher) throws IOException
|
||||||
|
{
|
||||||
|
if (dictionaryIdLookup.getLongBuffer() != null) {
|
||||||
|
writeInternal(smoosher, dictionaryIdLookup.getLongBuffer(), LONG_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
|
writeInternal(smoosher, dictionaryWriter, LONG_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -52,7 +52,6 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
|
||||||
protected final IndexSpec indexSpec;
|
protected final IndexSpec indexSpec;
|
||||||
@SuppressWarnings("unused")
|
@SuppressWarnings("unused")
|
||||||
protected final Closer closer;
|
protected final Closer closer;
|
||||||
protected final String dictionaryFileName;
|
|
||||||
|
|
||||||
protected DictionaryIdLookup dictionaryIdLookup;
|
protected DictionaryIdLookup dictionaryIdLookup;
|
||||||
protected DictionaryWriter<T> dictionaryWriter;
|
protected DictionaryWriter<T> dictionaryWriter;
|
||||||
|
@ -66,18 +65,15 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
|
||||||
|
|
||||||
public ScalarNestedCommonFormatColumnSerializer(
|
public ScalarNestedCommonFormatColumnSerializer(
|
||||||
String name,
|
String name,
|
||||||
String dictionaryFileName,
|
|
||||||
IndexSpec indexSpec,
|
IndexSpec indexSpec,
|
||||||
SegmentWriteOutMedium segmentWriteOutMedium,
|
SegmentWriteOutMedium segmentWriteOutMedium,
|
||||||
Closer closer
|
Closer closer
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.name = name;
|
this.name = name;
|
||||||
this.dictionaryFileName = dictionaryFileName;
|
|
||||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||||
this.indexSpec = indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
this.dictionaryIdLookup = new DictionaryIdLookup();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -99,6 +95,8 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
|
||||||
*/
|
*/
|
||||||
protected abstract void writeValueColumn(FileSmoosher smoosher) throws IOException;
|
protected abstract void writeValueColumn(FileSmoosher smoosher) throws IOException;
|
||||||
|
|
||||||
|
protected abstract void writeDictionaryFile(FileSmoosher smoosher) throws IOException;
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String getColumnName()
|
public String getColumnName()
|
||||||
{
|
{
|
||||||
|
@ -220,7 +218,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
|
||||||
}
|
}
|
||||||
|
|
||||||
writeV0Header(channel, columnNameBytes);
|
writeV0Header(channel, columnNameBytes);
|
||||||
writeInternal(smoosher, dictionaryWriter, dictionaryFileName);
|
writeDictionaryFile(smoosher);
|
||||||
writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME);
|
writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME);
|
||||||
writeValueColumn(smoosher);
|
writeValueColumn(smoosher);
|
||||||
writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME);
|
writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME);
|
||||||
|
|
|
@ -21,20 +21,17 @@ package org.apache.druid.segment.nested;
|
||||||
|
|
||||||
import com.google.common.base.Supplier;
|
import com.google.common.base.Supplier;
|
||||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
|
||||||
import org.apache.druid.java.util.common.RE;
|
import org.apache.druid.java.util.common.RE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.segment.column.ColumnBuilder;
|
import org.apache.druid.segment.column.ColumnBuilder;
|
||||||
import org.apache.druid.segment.column.ColumnConfig;
|
import org.apache.druid.segment.column.ColumnConfig;
|
||||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||||
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn;
|
||||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||||
import org.apache.druid.segment.data.ColumnarInts;
|
import org.apache.druid.segment.data.ColumnarInts;
|
||||||
import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
|
import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
|
||||||
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
|
|
||||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
import org.apache.druid.segment.data.Indexed;
|
import org.apache.druid.segment.data.Indexed;
|
||||||
import org.apache.druid.segment.data.VByte;
|
import org.apache.druid.segment.data.VByte;
|
||||||
|
@ -71,39 +68,11 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
|
||||||
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
||||||
);
|
);
|
||||||
|
|
||||||
final int dictionaryStartPosition = stringDictionaryBuffer.position();
|
dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
|
||||||
final byte dictionaryVersion = stringDictionaryBuffer.get();
|
mapper,
|
||||||
|
|
||||||
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
|
||||||
final byte encodingId = stringDictionaryBuffer.get();
|
|
||||||
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
|
||||||
dictionarySupplier = FrontCodedIndexed.read(
|
|
||||||
stringDictionaryBuffer,
|
stringDictionaryBuffer,
|
||||||
byteOrder
|
byteOrder
|
||||||
);
|
);
|
||||||
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
|
||||||
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
|
||||||
// this provides backwards compatibility should we switch at some point in the future to always
|
|
||||||
// writing dictionaryVersion
|
|
||||||
dictionarySupplier = GenericIndexed.read(
|
|
||||||
stringDictionaryBuffer,
|
|
||||||
GenericIndexed.UTF8_STRATEGY,
|
|
||||||
mapper
|
|
||||||
)::singleThreaded;
|
|
||||||
} else {
|
|
||||||
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading
|
|
||||||
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
|
||||||
// GenericIndexed version can be correctly read
|
|
||||||
stringDictionaryBuffer.position(dictionaryStartPosition);
|
|
||||||
dictionarySupplier = GenericIndexed.read(
|
|
||||||
stringDictionaryBuffer,
|
|
||||||
GenericIndexed.UTF8_STRATEGY,
|
|
||||||
mapper
|
|
||||||
)::singleThreaded;
|
|
||||||
}
|
|
||||||
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
||||||
mapper,
|
mapper,
|
||||||
columnName,
|
columnName,
|
||||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.druid.common.config.NullHandling;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.io.Closer;
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||||
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.math.expr.ExpressionType;
|
import org.apache.druid.math.expr.ExpressionType;
|
||||||
import org.apache.druid.segment.IndexSpec;
|
import org.apache.druid.segment.IndexSpec;
|
||||||
|
@ -46,7 +47,7 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
Closer closer
|
Closer closer
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(name, STRING_DICTIONARY_FILE_NAME, indexSpec, segmentWriteOutMedium, closer);
|
super(name, indexSpec, segmentWriteOutMedium, closer);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -67,6 +68,15 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
name
|
name
|
||||||
);
|
);
|
||||||
dictionaryWriter.open();
|
dictionaryWriter.open();
|
||||||
|
dictionaryIdLookup = closer.register(
|
||||||
|
new DictionaryIdLookup(
|
||||||
|
name,
|
||||||
|
dictionaryWriter,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -89,7 +99,6 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
|
|
||||||
// null is always 0
|
// null is always 0
|
||||||
dictionaryWriter.write(null);
|
dictionaryWriter.write(null);
|
||||||
dictionaryIdLookup.addString(null);
|
|
||||||
for (String value : strings) {
|
for (String value : strings) {
|
||||||
value = NullHandling.emptyToNullIfNeeded(value);
|
value = NullHandling.emptyToNullIfNeeded(value);
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
|
@ -97,7 +106,6 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
}
|
}
|
||||||
|
|
||||||
dictionaryWriter.write(value);
|
dictionaryWriter.write(value);
|
||||||
dictionaryIdLookup.addString(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
}
|
}
|
||||||
|
@ -107,4 +115,17 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
|
||||||
{
|
{
|
||||||
// no extra value column for strings
|
// no extra value column for strings
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void writeDictionaryFile(FileSmoosher smoosher) throws IOException
|
||||||
|
{
|
||||||
|
if (dictionaryIdLookup.getStringBufferMapper() != null) {
|
||||||
|
SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper();
|
||||||
|
for (String name : fileMapper.getInternalFilenames()) {
|
||||||
|
smoosher.add(name, fileMapper.mapFile(name));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.google.common.base.Supplier;
|
||||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||||
import org.apache.druid.error.DruidException;
|
import org.apache.druid.error.DruidException;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
|
||||||
import org.apache.druid.java.util.common.RE;
|
import org.apache.druid.java.util.common.RE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
|
@ -35,15 +34,13 @@ import org.apache.druid.segment.column.ColumnBuilder;
|
||||||
import org.apache.druid.segment.column.ColumnConfig;
|
import org.apache.druid.segment.column.ColumnConfig;
|
||||||
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
import org.apache.druid.segment.column.ColumnIndexSupplier;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||||
import org.apache.druid.segment.column.TypeSignature;
|
import org.apache.druid.segment.column.TypeSignature;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||||
import org.apache.druid.segment.data.ColumnarInts;
|
import org.apache.druid.segment.data.ColumnarInts;
|
||||||
import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
|
import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
|
||||||
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
|
|
||||||
import org.apache.druid.segment.data.FixedIndexed;
|
import org.apache.druid.segment.data.FixedIndexed;
|
||||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
|
||||||
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
import org.apache.druid.segment.data.Indexed;
|
import org.apache.druid.segment.data.Indexed;
|
||||||
|
@ -92,8 +89,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
if (version == NestedCommonFormatColumnSerializer.V0) {
|
if (version == NestedCommonFormatColumnSerializer.V0) {
|
||||||
try {
|
try {
|
||||||
final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
|
final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
|
||||||
final GenericIndexed<ByteBuffer> stringDictionary;
|
final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
|
||||||
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
|
|
||||||
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
|
||||||
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
|
||||||
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
|
||||||
|
@ -105,34 +101,11 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
|
||||||
);
|
);
|
||||||
|
|
||||||
final int dictionaryStartPosition = stringDictionaryBuffer.position();
|
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
|
||||||
final byte dictionaryVersion = stringDictionaryBuffer.get();
|
mapper,
|
||||||
|
|
||||||
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
|
||||||
final byte encodingId = stringDictionaryBuffer.get();
|
|
||||||
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
|
||||||
frontCodedStringDictionarySupplier = FrontCodedIndexed.read(
|
|
||||||
stringDictionaryBuffer,
|
stringDictionaryBuffer,
|
||||||
byteOrder
|
byteOrder
|
||||||
);
|
);
|
||||||
stringDictionary = null;
|
|
||||||
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
|
||||||
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
|
||||||
// this provides backwards compatibility should we switch at some point in the future to always
|
|
||||||
// writing dictionaryVersion
|
|
||||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
|
||||||
frontCodedStringDictionarySupplier = null;
|
|
||||||
} else {
|
|
||||||
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading
|
|
||||||
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
|
||||||
// GenericIndexed version can be correctly read
|
|
||||||
stringDictionaryBuffer.position(dictionaryStartPosition);
|
|
||||||
stringDictionary = GenericIndexed.read(stringDictionaryBuffer, GenericIndexed.UTF8_STRATEGY, mapper);
|
|
||||||
frontCodedStringDictionarySupplier = null;
|
|
||||||
}
|
|
||||||
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
|
||||||
mapper,
|
mapper,
|
||||||
columnName,
|
columnName,
|
||||||
|
@ -213,8 +186,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
return new VariantColumnAndIndexSupplier(
|
return new VariantColumnAndIndexSupplier(
|
||||||
logicalType,
|
logicalType,
|
||||||
variantTypeByte,
|
variantTypeByte,
|
||||||
stringDictionary,
|
stringDictionarySupplier,
|
||||||
frontCodedStringDictionarySupplier,
|
|
||||||
longDictionarySupplier,
|
longDictionarySupplier,
|
||||||
doubleDictionarySupplier,
|
doubleDictionarySupplier,
|
||||||
arrayDictionarySupplier,
|
arrayDictionarySupplier,
|
||||||
|
@ -255,8 +227,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
public VariantColumnAndIndexSupplier(
|
public VariantColumnAndIndexSupplier(
|
||||||
ColumnType logicalType,
|
ColumnType logicalType,
|
||||||
@Nullable Byte variantTypeSetByte,
|
@Nullable Byte variantTypeSetByte,
|
||||||
GenericIndexed<ByteBuffer> stringDictionary,
|
Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier,
|
||||||
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
|
|
||||||
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
Supplier<FixedIndexed<Long>> longDictionarySupplier,
|
||||||
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
|
||||||
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
|
||||||
|
@ -271,9 +242,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
|
||||||
{
|
{
|
||||||
this.logicalType = logicalType;
|
this.logicalType = logicalType;
|
||||||
this.variantTypeSetByte = variantTypeSetByte;
|
this.variantTypeSetByte = variantTypeSetByte;
|
||||||
stringDictionarySupplier = frontCodedStringDictionarySupplier != null
|
this.stringDictionarySupplier = stringDictionarySupplier;
|
||||||
? frontCodedStringDictionarySupplier
|
|
||||||
: stringDictionary::singleThreaded;
|
|
||||||
this.longDictionarySupplier = longDictionarySupplier;
|
this.longDictionarySupplier = longDictionarySupplier;
|
||||||
this.doubleDictionarySupplier = doubleDictionarySupplier;
|
this.doubleDictionarySupplier = doubleDictionarySupplier;
|
||||||
this.arrayDictionarySupplier = arrayDictionarySupplier;
|
this.arrayDictionarySupplier = arrayDictionarySupplier;
|
||||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.StringUtils;
|
import org.apache.druid.java.util.common.StringUtils;
|
||||||
import org.apache.druid.java.util.common.io.Closer;
|
import org.apache.druid.java.util.common.io.Closer;
|
||||||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||||
|
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.math.expr.ExprEval;
|
import org.apache.druid.math.expr.ExprEval;
|
||||||
import org.apache.druid.segment.ColumnValueSelector;
|
import org.apache.druid.segment.ColumnValueSelector;
|
||||||
|
@ -94,7 +95,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
this.segmentWriteOutMedium = segmentWriteOutMedium;
|
||||||
this.indexSpec = indexSpec;
|
this.indexSpec = indexSpec;
|
||||||
this.closer = closer;
|
this.closer = closer;
|
||||||
this.dictionaryIdLookup = new DictionaryIdLookup();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -151,6 +151,15 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
arrayDictionaryWriter.open();
|
arrayDictionaryWriter.open();
|
||||||
arrayElementDictionaryWriter = new FixedIndexedIntWriter(segmentWriteOutMedium, true);
|
arrayElementDictionaryWriter = new FixedIndexedIntWriter(segmentWriteOutMedium, true);
|
||||||
arrayElementDictionaryWriter.open();
|
arrayElementDictionaryWriter.open();
|
||||||
|
dictionaryIdLookup = closer.register(
|
||||||
|
new DictionaryIdLookup(
|
||||||
|
name,
|
||||||
|
dictionaryWriter,
|
||||||
|
longDictionaryWriter,
|
||||||
|
doubleDictionaryWriter,
|
||||||
|
arrayDictionaryWriter
|
||||||
|
)
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -177,7 +186,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
|
|
||||||
// null is always 0
|
// null is always 0
|
||||||
dictionaryWriter.write(null);
|
dictionaryWriter.write(null);
|
||||||
dictionaryIdLookup.addString(null);
|
|
||||||
for (String value : strings) {
|
for (String value : strings) {
|
||||||
value = NullHandling.emptyToNullIfNeeded(value);
|
value = NullHandling.emptyToNullIfNeeded(value);
|
||||||
if (value == null) {
|
if (value == null) {
|
||||||
|
@ -185,7 +193,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
}
|
}
|
||||||
|
|
||||||
dictionaryWriter.write(value);
|
dictionaryWriter.write(value);
|
||||||
dictionaryIdLookup.addString(value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (Long value : longs) {
|
for (Long value : longs) {
|
||||||
|
@ -193,7 +200,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
longDictionaryWriter.write(value);
|
longDictionaryWriter.write(value);
|
||||||
dictionaryIdLookup.addLong(value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (Double value : doubles) {
|
for (Double value : doubles) {
|
||||||
|
@ -201,7 +207,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
doubleDictionaryWriter.write(value);
|
doubleDictionaryWriter.write(value);
|
||||||
dictionaryIdLookup.addDouble(value);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
for (int[] value : arrays) {
|
for (int[] value : arrays) {
|
||||||
|
@ -209,7 +214,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
arrayDictionaryWriter.write(value);
|
arrayDictionaryWriter.write(value);
|
||||||
dictionaryIdLookup.addArray(value);
|
|
||||||
}
|
}
|
||||||
dictionarySerialized = true;
|
dictionarySerialized = true;
|
||||||
}
|
}
|
||||||
|
@ -378,10 +382,30 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
|
||||||
channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte}));
|
channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte}));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (dictionaryIdLookup.getStringBufferMapper() != null) {
|
||||||
|
SmooshedFileMapper fileMapper = dictionaryIdLookup.getStringBufferMapper();
|
||||||
|
for (String internalName : fileMapper.getInternalFilenames()) {
|
||||||
|
smoosher.add(internalName, fileMapper.mapFile(internalName));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
if (dictionaryIdLookup.getLongBuffer() != null) {
|
||||||
|
writeInternal(smoosher, dictionaryIdLookup.getLongBuffer(), LONG_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
if (dictionaryIdLookup.getDoubleBuffer() != null) {
|
||||||
|
writeInternal(smoosher, dictionaryIdLookup.getDoubleBuffer(), DOUBLE_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
if (dictionaryIdLookup.getArrayBuffer() != null) {
|
||||||
|
writeInternal(smoosher, dictionaryIdLookup.getArrayBuffer(), ARRAY_DICTIONARY_FILE_NAME);
|
||||||
|
} else {
|
||||||
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
|
||||||
|
}
|
||||||
|
|
||||||
writeInternal(smoosher, arrayElementDictionaryWriter, ARRAY_ELEMENT_DICTIONARY_FILE_NAME);
|
writeInternal(smoosher, arrayElementDictionaryWriter, ARRAY_ELEMENT_DICTIONARY_FILE_NAME);
|
||||||
writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME);
|
writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME);
|
||||||
writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME);
|
writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME);
|
||||||
|
|
|
@ -28,11 +28,10 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||||
import org.apache.druid.collections.spatial.ImmutableRTree;
|
import org.apache.druid.collections.spatial.ImmutableRTree;
|
||||||
import org.apache.druid.io.Channels;
|
import org.apache.druid.io.Channels;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
|
||||||
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
|
||||||
import org.apache.druid.segment.column.ColumnBuilder;
|
import org.apache.druid.segment.column.ColumnBuilder;
|
||||||
import org.apache.druid.segment.column.ColumnConfig;
|
import org.apache.druid.segment.column.ColumnConfig;
|
||||||
import org.apache.druid.segment.column.StringEncodingStrategy;
|
import org.apache.druid.segment.column.StringEncodingStrategies;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
import org.apache.druid.segment.data.BitmapSerde;
|
import org.apache.druid.segment.data.BitmapSerde;
|
||||||
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
import org.apache.druid.segment.data.BitmapSerdeFactory;
|
||||||
|
@ -43,8 +42,6 @@ import org.apache.druid.segment.data.ColumnarMultiInts;
|
||||||
import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
|
import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier;
|
||||||
import org.apache.druid.segment.data.CompressedVSizeColumnarMultiIntsSupplier;
|
import org.apache.druid.segment.data.CompressedVSizeColumnarMultiIntsSupplier;
|
||||||
import org.apache.druid.segment.data.DictionaryWriter;
|
import org.apache.druid.segment.data.DictionaryWriter;
|
||||||
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
|
|
||||||
import org.apache.druid.segment.data.FrontCodedIndexed;
|
|
||||||
import org.apache.druid.segment.data.GenericIndexed;
|
import org.apache.druid.segment.data.GenericIndexed;
|
||||||
import org.apache.druid.segment.data.GenericIndexedWriter;
|
import org.apache.druid.segment.data.GenericIndexedWriter;
|
||||||
import org.apache.druid.segment.data.ImmutableRTreeObjectStrategy;
|
import org.apache.druid.segment.data.ImmutableRTreeObjectStrategy;
|
||||||
|
@ -312,37 +309,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
|
||||||
|
|
||||||
builder.setType(ValueType.STRING);
|
builder.setType(ValueType.STRING);
|
||||||
|
|
||||||
final int dictionaryStartPosition = buffer.position();
|
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier =
|
||||||
final byte dictionaryVersion = buffer.get();
|
StringEncodingStrategies.getStringDictionarySupplier(
|
||||||
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
|
builder.getFileMapper(),
|
||||||
|
|
||||||
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
|
|
||||||
final byte encodingId = buffer.get();
|
|
||||||
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
|
|
||||||
dictionarySupplier = FrontCodedIndexed.read(buffer, byteOrder);
|
|
||||||
} else if (encodingId == StringEncodingStrategy.UTF8_ID) {
|
|
||||||
// this cannot happen naturally right now since generic indexed is written in the 'legacy' format, but
|
|
||||||
// this provides backwards compatibility should we switch at some point in the future to always
|
|
||||||
// writing dictionaryVersion
|
|
||||||
dictionarySupplier = GenericIndexed.read(
|
|
||||||
buffer,
|
buffer,
|
||||||
GenericIndexed.UTF8_STRATEGY,
|
byteOrder
|
||||||
builder.getFileMapper()
|
);
|
||||||
)::singleThreaded;
|
|
||||||
} else {
|
|
||||||
throw new ISE("impossible, unknown encoding strategy id: %s", encodingId);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
// legacy format that only supports plain utf8 enoding stored in GenericIndexed and the byte we are reading
|
|
||||||
// as dictionaryVersion is actually also the GenericIndexed version, so we reset start position so the
|
|
||||||
// GenericIndexed version can be correctly read
|
|
||||||
buffer.position(dictionaryStartPosition);
|
|
||||||
dictionarySupplier = GenericIndexed.read(
|
|
||||||
buffer,
|
|
||||||
GenericIndexed.UTF8_STRATEGY,
|
|
||||||
builder.getFileMapper()
|
|
||||||
)::singleThreaded;
|
|
||||||
}
|
|
||||||
|
|
||||||
final WritableSupplier<ColumnarInts> rSingleValuedColumn;
|
final WritableSupplier<ColumnarInts> rSingleValuedColumn;
|
||||||
final WritableSupplier<ColumnarMultiInts> rMultiValuedColumn;
|
final WritableSupplier<ColumnarMultiInts> rMultiValuedColumn;
|
||||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema;
|
||||||
import org.apache.druid.segment.incremental.IndexSizeExceededException;
|
import org.apache.druid.segment.incremental.IndexSizeExceededException;
|
||||||
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
|
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
|
||||||
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
|
||||||
|
import org.apache.druid.testing.InitializedNullHandlingTest;
|
||||||
import org.apache.druid.timeline.SegmentId;
|
import org.apache.druid.timeline.SegmentId;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
@ -81,7 +82,7 @@ import java.util.stream.IntStream;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class DoubleStorageTest
|
public class DoubleStorageTest extends InitializedNullHandlingTest
|
||||||
{
|
{
|
||||||
|
|
||||||
private static final SegmentMetadataQueryRunnerFactory METADATA_QR_FACTORY = new SegmentMetadataQueryRunnerFactory(
|
private static final SegmentMetadataQueryRunnerFactory METADATA_QR_FACTORY = new SegmentMetadataQueryRunnerFactory(
|
||||||
|
|
|
@ -141,6 +141,35 @@ public class FixedIndexedTest extends InitializedNullHandlingTest
|
||||||
for (Long aLong : LONGS) {
|
for (Long aLong : LONGS) {
|
||||||
writer.write(aLong);
|
writer.write(aLong);
|
||||||
}
|
}
|
||||||
|
Iterator<Long> longIterator = writer.getIterator();
|
||||||
|
int ctr = 0;
|
||||||
|
int totalCount = withNull ? 1 + LONGS.length : LONGS.length;
|
||||||
|
for (int i = 0; i < totalCount; i++) {
|
||||||
|
if (withNull) {
|
||||||
|
if (i == 0) {
|
||||||
|
Assert.assertNull(writer.get(i));
|
||||||
|
} else {
|
||||||
|
Assert.assertEquals(" index: " + i, LONGS[i - 1], writer.get(i));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
Assert.assertEquals(" index: " + i, LONGS[i], writer.get(i));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
while (longIterator.hasNext()) {
|
||||||
|
if (withNull) {
|
||||||
|
if (ctr == 0) {
|
||||||
|
Assert.assertNull(longIterator.next());
|
||||||
|
Assert.assertNull(writer.get(ctr));
|
||||||
|
} else {
|
||||||
|
Assert.assertEquals(LONGS[ctr - 1], longIterator.next());
|
||||||
|
Assert.assertEquals(LONGS[ctr - 1], writer.get(ctr));
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
Assert.assertEquals(LONGS[ctr], longIterator.next());
|
||||||
|
Assert.assertEquals(LONGS[ctr], writer.get(ctr));
|
||||||
|
}
|
||||||
|
ctr++;
|
||||||
|
}
|
||||||
WritableByteChannel channel = new WritableByteChannel()
|
WritableByteChannel channel = new WritableByteChannel()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
Loading…
Reference in New Issue