use mmap for nested column value to dictionary id lookup for more chill heap usage during serialization (#14919)

This commit is contained in:
Clint Wylie 2023-09-12 21:01:18 -07:00 committed by GitHub
parent 286eecad7c
commit 23b78c0f95
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 571 additions and 405 deletions

View File

@ -228,13 +228,12 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
} else {
// all the bells and whistles
logicalType = ColumnType.NESTED_DATA;
final NestedDataColumnSerializer defaultSerializer = new NestedDataColumnSerializer(
serializer = new NestedDataColumnSerializer(
name,
indexSpec,
segmentWriteOutMedium,
closer
);
serializer = defaultSerializer;
}
serializer.openDictionaryWriter();

View File

@ -57,7 +57,7 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
private final Closer closer;
private ColumnDescriptor.Builder descriptorBuilder;
private GenericColumnSerializer<?> serializer;
private NestedDataColumnSerializerV4 serializer;
public NestedDataColumnMergerV4(
String name,
@ -111,13 +111,12 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
descriptorBuilder = new ColumnDescriptor.Builder();
final NestedDataColumnSerializerV4 defaultSerializer = new NestedDataColumnSerializerV4(
serializer = new NestedDataColumnSerializerV4(
name,
indexSpec,
segmentWriteOutMedium,
closer
);
serializer = defaultSerializer;
final ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.serializerBuilder()
.withTypeName(NestedDataComplexTypeSerde.TYPE_NAME)
@ -127,14 +126,14 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
.setHasMultipleValues(false)
.addSerde(partSerde);
defaultSerializer.open();
defaultSerializer.serializeFields(mergedFields);
serializer.open();
serializer.serializeFields(mergedFields);
int stringCardinality;
int longCardinality;
int doubleCardinality;
if (numMergeIndex == 1) {
defaultSerializer.serializeDictionaries(
serializer.serializeDictionaries(
sortedLookup.getSortedStrings(),
sortedLookup.getSortedLongs(),
sortedLookup.getSortedDoubles()
@ -155,7 +154,7 @@ public class NestedDataColumnMergerV4 implements DimensionMergerV9
sortedDoubleLookups,
DOUBLE_MERGING_COMPARATOR
);
defaultSerializer.serializeDictionaries(
serializer.serializeDictionaries(
() -> stringIterator,
() -> longIterator,
() -> doubleIterator

View File

@ -19,12 +19,15 @@
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.StringUtils;
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.IndexIO;
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.FrontCodedIndexedWriter;
import org.apache.druid.segment.data.GenericIndexed;
import org.apache.druid.segment.data.GenericIndexedWriter;
@ -33,6 +36,7 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import javax.annotation.Nullable;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.Iterator;
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
* to consumers.

View File

@ -155,7 +155,7 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
if (index == 0 && hasNulls) {
return null;
}
int startOffset = index * width;
int startOffset = (hasNulls ? index - 1 : index) * width;
readBuffer.clear();
valuesOut.readFully(startOffset, readBuffer);
readBuffer.clear();
@ -197,14 +197,14 @@ public class FixedIndexedWriter<T> implements DictionaryWriter<T>
{
iteratorBuffer.clear();
try {
if (totalCount - pos < PAGE_SIZE) {
int size = (totalCount - pos) * width;
if (numWritten - (pos - startPos) < PAGE_SIZE) {
int size = (numWritten - (pos - startPos)) * width;
iteratorBuffer.limit(size);
valuesOut.readFully((long) pos * width, iteratorBuffer);
valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer);
} else {
valuesOut.readFully((long) pos * width, iteratorBuffer);
valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer);
}
iteratorBuffer.flip();
iteratorBuffer.clear();
}
catch (IOException e) {
throw new RuntimeException(e);

View File

@ -234,6 +234,7 @@ public class FrontCodedIndexedWriter implements DictionaryWriter<byte[]>
{
getOffsetBuffer.clear();
headerOut.readFully(index * (long) Integer.BYTES, getOffsetBuffer);
getOffsetBuffer.clear();
return getOffsetBuffer.getInt(0);
}

View File

@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
import com.google.common.primitives.Ints;
import it.unimi.dsi.fastutil.longs.LongArrayList;
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.ISE;
import org.apache.druid.java.util.common.StringUtils;
@ -293,7 +294,16 @@ public class GenericIndexedWriter<T> implements DictionaryWriter<T>
long endOffset = getOffset(index);
int valueSize = checkedCastNonnegativeLongToInt(endOffset - startOffset);
if (valueSize == 0) {
return null;
if (NullHandling.replaceWithDefault()) {
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);
valuesOut.readFully(startOffset, bb);

View File

@ -19,114 +19,308 @@
package org.apache.druid.segment.nested;
import com.google.common.base.Preconditions;
import it.unimi.dsi.fastutil.doubles.Double2IntLinkedOpenHashMap;
import it.unimi.dsi.fastutil.doubles.Double2IntMap;
import it.unimi.dsi.fastutil.longs.Long2IntLinkedOpenHashMap;
import it.unimi.dsi.fastutil.longs.Long2IntMap;
import it.unimi.dsi.fastutil.objects.Object2IntAVLTreeMap;
import it.unimi.dsi.fastutil.objects.Object2IntLinkedOpenHashMap;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import org.apache.druid.segment.data.FrontCodedIntArrayIndexedWriter;
import com.google.common.primitives.Ints;
import org.apache.druid.annotations.SuppressFBWarnings;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.ByteBufferUtils;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.StringUtils;
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.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 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
* dictionary id to value mapping for the 'stacked' global value dictionaries.
* Value to dictionary id lookup, backed with memory mapped dictionaries populated lazily by the supplied
* @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()
public DictionaryIdLookup(
String name,
@Nullable DictionaryWriter<String> stringDictionaryWriter,
@Nullable DictionaryWriter<Long> longDictionaryWriter,
@Nullable DictionaryWriter<Double> doubleDictionaryWriter,
@Nullable DictionaryWriter<int[]> arrayDictionaryWriter
)
{
this.stringLookup = new Object2IntLinkedOpenHashMap<>();
stringLookup.defaultReturnValue(-1);
this.longLookup = new Long2IntLinkedOpenHashMap();
longLookup.defaultReturnValue(-1);
this.doubleLookup = new Double2IntLinkedOpenHashMap();
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++;
this.name = name;
this.stringDictionaryWriter = stringDictionaryWriter;
this.longDictionaryWriter = longDictionaryWriter;
this.doubleDictionaryWriter = doubleDictionaryWriter;
this.arrayDictionaryWriter = arrayDictionaryWriter;
}
public int lookupString(@Nullable String value)
{
return stringLookup.getInt(value);
}
public void addLong(long value)
{
Preconditions.checkState(
doubleLookup.size() == 0,
"All long values must be inserted to the lookup before double types"
);
int id = dictionarySize++;
longLookup.put(value, id);
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
// 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)
File stringSmoosh = FileUtils.createTempDir(name + "__stringTempSmoosh");
final String fileName = NestedCommonFormatColumnSerializer.getInternalFileName(
name,
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
);
final FileSmoosher smoosher = new FileSmoosher(stringSmoosh);
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)
{
if (value == null) {
return 0;
if (longDictionary == null) {
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());
}
public void addDouble(double value)
{
int id = dictionarySize++;
doubleLookup.put(value, id);
final int index = longDictionary.indexOf(value);
if (index < 0) {
throw DruidException.defensive("Value not found in long dictionary");
}
return index + longOffset();
}
public int lookupDouble(@Nullable Double value)
{
if (value == null) {
return 0;
if (doubleDictionary == null) {
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());
}
public void addArray(int[] value)
{
int id = dictionarySize++;
arrayLookup.put(value, id);
final int index = doubleDictionary.indexOf(value);
if (index < 0) {
throw DruidException.defensive("Value not found in double dictionary");
}
return index + doubleOffset();
}
public int lookupArray(@Nullable int[] value)
{
if (value == null) {
return 0;
if (arrayDictionary == null) {
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);
}
};
}
}

View File

@ -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
{
channel.write(ByteBuffer.wrap(new byte[]{NestedCommonFormatColumnSerializer.V0}));

View File

@ -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.io.Closer;
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.logger.Logger;
import org.apache.druid.math.expr.ExprEval;
@ -170,7 +171,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.indexSpec = indexSpec;
this.closer = closer;
this.globalDictionaryIdLookup = new DictionaryIdLookup();
}
@Override
@ -231,6 +231,15 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
4
);
arrayDictionaryWriter.open();
globalDictionaryIdLookup = closer.register(
new DictionaryIdLookup(
name,
dictionaryWriter,
longDictionaryWriter,
doubleDictionaryWriter,
arrayDictionaryWriter
)
);
}
@Override
@ -329,7 +338,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
// null is always 0
dictionaryWriter.write(null);
globalDictionaryIdLookup.addString(null);
for (String value : strings) {
value = NullHandling.emptyToNullIfNeeded(value);
if (value == null) {
@ -337,7 +345,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
}
dictionaryWriter.write(value);
globalDictionaryIdLookup.addString(value);
}
dictionarySerialized = true;
@ -346,7 +353,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
continue;
}
longDictionaryWriter.write(value);
globalDictionaryIdLookup.addLong(value);
}
for (Double value : doubles) {
@ -354,7 +360,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
continue;
}
doubleDictionaryWriter.write(value);
globalDictionaryIdLookup.addDouble(value);
}
for (int[] value : arrays) {
@ -362,7 +367,6 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
continue;
}
arrayDictionaryWriter.write(value);
globalDictionaryIdLookup.addArray(value);
}
dictionarySerialized = true;
}
@ -433,10 +437,31 @@ public class NestedDataColumnSerializer extends NestedCommonFormatColumnSerializ
writeV0Header(channel, columnNameBytes);
fieldsWriter.writeTo(channel, smoosher);
fieldsInfoWriter.writeTo(channel, smoosher);
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
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);
}
if (globalDictionaryIdLookup.getLongBuffer() != null) {
writeInternal(smoosher, globalDictionaryIdLookup.getLongBuffer(), LONG_DICTIONARY_FILE_NAME);
} else {
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);
}
if (globalDictionaryIdLookup.getArrayBuffer() != null) {
writeInternal(smoosher, globalDictionaryIdLookup.getArrayBuffer(), ARRAY_DICTIONARY_FILE_NAME);
} else {
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
}
writeInternal(smoosher, rawWriter, RAW_FILE_NAME);
if (!nullRowsBitmap.isEmpty()) {
writeInternal(smoosher, nullBitmapWriter, NULL_BITMAP_FILE_NAME);

View File

@ -144,7 +144,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.indexSpec = indexSpec;
this.closer = closer;
this.globalDictionaryIdLookup = new DictionaryIdLookup();
}
@Override
@ -195,6 +194,16 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
nullBitmapWriter.open();
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
@ -263,7 +272,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
// null is always 0
dictionaryWriter.write(null);
globalDictionaryIdLookup.addString(null);
for (String value : strings) {
value = NullHandling.emptyToNullIfNeeded(value);
if (value == null) {
@ -271,7 +279,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
}
dictionaryWriter.write(value);
globalDictionaryIdLookup.addString(value);
}
dictionarySerialized = true;
@ -280,7 +287,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
continue;
}
longDictionaryWriter.write(value);
globalDictionaryIdLookup.addLong(value);
}
for (Double value : doubles) {
@ -288,7 +294,6 @@ public class NestedDataColumnSerializerV4 implements GenericColumnSerializer<Str
continue;
}
doubleDictionaryWriter.write(value);
globalDictionaryIdLookup.addDouble(value);
}
dictionarySerialized = true;
}

View File

@ -21,21 +21,19 @@ package org.apache.druid.segment.nested;
import com.google.common.base.Supplier;
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.StringUtils;
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnConfig;
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.CompressedVariableSizedBlobColumnSupplier;
import org.apache.druid.segment.data.EncodedStringDictionaryWriter;
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.GenericIndexed;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.data.VByte;
import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde;
@ -67,8 +65,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
final FieldTypeInfo fieldInfo;
final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
final ImmutableBitmap nullValues;
final GenericIndexed<ByteBuffer> stringDictionary;
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
@ -82,34 +79,12 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
);
final int dictionaryStartPosition = stringDictionaryBuffer.position();
final byte dictionaryVersion = stringDictionaryBuffer.get();
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
mapper,
stringDictionaryBuffer,
byteOrder
);
if (dictionaryVersion == EncodedStringDictionaryWriter.VERSION) {
final byte encodingId = stringDictionaryBuffer.get();
if (encodingId == StringEncodingStrategy.FRONT_CODED_ID) {
frontCodedStringDictionarySupplier = FrontCodedIndexed.read(
stringDictionaryBuffer,
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(
mapper,
columnName,
@ -173,8 +148,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
fieldInfo,
compressedRawColumnSupplier,
nullValues,
stringDictionary,
frontCodedStringDictionarySupplier,
stringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
arrayDictionarySupplier,
@ -198,8 +172,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
private final FieldTypeInfo fieldInfo;
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
private final ImmutableBitmap nullValues;
private final GenericIndexed<ByteBuffer> stringDictionary;
private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
private final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
@ -217,8 +190,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
FieldTypeInfo fieldInfo,
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
ImmutableBitmap nullValues,
GenericIndexed<ByteBuffer> stringDictionary,
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
@ -234,8 +206,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
this.fieldInfo = fieldInfo;
this.compressedRawColumnSupplier = compressedRawColumnSupplier;
this.nullValues = nullValues;
this.stringDictionary = stringDictionary;
this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier;
this.stringDictionarySupplier = stringDictionarySupplier;
this.longDictionarySupplier = longDictionarySupplier;
this.doubleDictionarySupplier = doubleDictionarySupplier;
this.arrayDictionarySupplier = arrayDictionarySupplier;
@ -249,24 +220,6 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
@Override
public NestedCommonFormatColumn get()
{
if (frontCodedStringDictionarySupplier != null) {
return new NestedDataColumnV5<>(
columnName,
getLogicalType(),
columnConfig,
compressedRawColumnSupplier,
nullValues,
fields,
fieldInfo,
frontCodedStringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
arrayDictionarySupplier,
fileMapper,
bitmapSerdeFactory,
byteOrder
);
}
return new NestedDataColumnV5<>(
columnName,
getLogicalType(),
@ -275,7 +228,7 @@ public class NestedDataColumnSupplier implements Supplier<NestedCommonFormatColu
nullValues,
fields,
fieldInfo,
stringDictionary::singleThreaded,
stringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
arrayDictionarySupplier,

View File

@ -22,7 +22,6 @@ package org.apache.druid.segment.nested;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Supplier;
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.io.smoosh.SmooshedFileMapper;
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.ColumnType;
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.data.BitmapSerdeFactory;
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.FrontCodedIndexed;
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
import org.apache.druid.segment.data.GenericIndexed;
import org.apache.druid.segment.data.Indexed;
import javax.annotation.Nullable;
import java.io.IOException;
@ -84,8 +82,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
final FieldTypeInfo fieldInfo;
final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
final ImmutableBitmap nullValues;
final GenericIndexed<ByteBuffer> stringDictionary;
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
@ -118,34 +115,11 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
);
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) {
frontCodedStringDictionarySupplier = FrontCodedIndexed.read(
stringDictionaryBuffer,
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;
}
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
mapper,
stringDictionaryBuffer,
metadata.getByteOrder()
);
final ByteBuffer longDictionaryBuffer = loadInternalFile(
mapper,
metadata,
@ -210,8 +184,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
fieldInfo,
compressedRawColumnSupplier,
nullValues,
stringDictionary,
frontCodedStringDictionarySupplier,
stringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
arrayDictionarySupplier,
@ -236,8 +209,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
private final FieldTypeInfo fieldInfo;
private final CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier;
private final ImmutableBitmap nullValues;
private final GenericIndexed<ByteBuffer> stringDictionary;
private final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
private final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
private final Supplier<FixedIndexed<Long>> longDictionarySupplier;
private final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
@ -257,8 +229,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
FieldTypeInfo fieldInfo,
CompressedVariableSizedBlobColumnSupplier compressedRawColumnSupplier,
ImmutableBitmap nullValues,
GenericIndexed<ByteBuffer> stringDictionary,
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
@ -275,8 +246,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
this.fieldInfo = fieldInfo;
this.compressedRawColumnSupplier = compressedRawColumnSupplier;
this.nullValues = nullValues;
this.stringDictionary = stringDictionary;
this.frontCodedStringDictionarySupplier = frontCodedStringDictionarySupplier;
this.stringDictionarySupplier = stringDictionarySupplier;
this.longDictionarySupplier = longDictionarySupplier;
this.doubleDictionarySupplier = doubleDictionarySupplier;
this.arrayDictionarySupplier = arrayDictionarySupplier;
@ -307,23 +277,6 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
private NestedDataColumnV3 makeV3()
{
if (frontCodedStringDictionarySupplier != null) {
return new NestedDataColumnV3<>(
columnName,
logicalType,
columnConfig,
compressedRawColumnSupplier,
nullValues,
fields,
fieldInfo,
frontCodedStringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
fileMapper,
bitmapSerdeFactory,
byteOrder
);
}
return new NestedDataColumnV3<>(
columnName,
logicalType,
@ -332,7 +285,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
nullValues,
fields,
fieldInfo,
stringDictionary::singleThreaded,
stringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
fileMapper,
@ -343,23 +296,6 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
private NestedDataColumnV4 makeV4()
{
if (frontCodedStringDictionarySupplier != null) {
return new NestedDataColumnV4<>(
columnName,
logicalType,
columnConfig,
compressedRawColumnSupplier,
nullValues,
fields,
fieldInfo,
frontCodedStringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
fileMapper,
bitmapSerdeFactory,
byteOrder
);
}
return new NestedDataColumnV4<>(
columnName,
logicalType,
@ -368,7 +304,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
nullValues,
fields,
fieldInfo,
stringDictionary::singleThreaded,
stringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
fileMapper,
@ -379,24 +315,6 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
private NestedDataColumnV5 makeV5()
{
if (frontCodedStringDictionarySupplier != null) {
return new NestedDataColumnV5<>(
columnName,
logicalType,
columnConfig,
compressedRawColumnSupplier,
nullValues,
fields,
fieldInfo,
frontCodedStringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
arrayDictionarySupplier,
fileMapper,
bitmapSerdeFactory,
byteOrder
);
}
return new NestedDataColumnV5<>(
columnName,
logicalType,
@ -405,7 +323,7 @@ public class NestedDataColumnSupplierV4 implements Supplier<ComplexColumn>
nullValues,
fields,
fieldInfo,
stringDictionary::singleThreaded,
stringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
arrayDictionarySupplier,

View File

@ -49,7 +49,7 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
Closer closer
)
{
super(name, DOUBLE_DICTIONARY_FILE_NAME, indexSpec, segmentWriteOutMedium, closer);
super(name, indexSpec, segmentWriteOutMedium, closer);
}
@Override
@ -73,6 +73,15 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
true
);
dictionaryWriter.open();
dictionaryIdLookup = closer.register(
new DictionaryIdLookup(
name,
null,
null,
dictionaryWriter,
null
)
);
}
@Override
@ -102,16 +111,15 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
// null is always 0
dictionaryWriter.write(null);
dictionaryIdLookup.addNumericNull();
for (Double value : doubles) {
if (value == null) {
continue;
}
dictionaryWriter.write(value);
dictionaryIdLookup.addDouble(value);
}
dictionarySerialized = true;
}
@Override
@ -119,4 +127,14 @@ public class ScalarDoubleColumnSerializer extends ScalarNestedCommonFormatColumn
{
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);
}
}
}

View File

@ -49,7 +49,7 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
Closer closer
)
{
super(name, LONG_DICTIONARY_FILE_NAME, indexSpec, segmentWriteOutMedium, closer);
super(name, indexSpec, segmentWriteOutMedium, closer);
}
@Override
@ -74,6 +74,15 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
true
);
dictionaryWriter.open();
dictionaryIdLookup = closer.register(
new DictionaryIdLookup(
name,
null,
dictionaryWriter,
null,
null
)
);
}
@Override
@ -104,14 +113,11 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
// null is always 0
dictionaryWriter.write(null);
dictionaryIdLookup.addNumericNull();
for (Long value : longs) {
if (value == null) {
continue;
}
dictionaryWriter.write(value);
dictionaryIdLookup.addLong(value);
}
dictionarySerialized = true;
}
@ -121,4 +127,14 @@ public class ScalarLongColumnSerializer extends ScalarNestedCommonFormatColumnSe
{
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);
}
}
}

View File

@ -52,7 +52,6 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
protected final IndexSpec indexSpec;
@SuppressWarnings("unused")
protected final Closer closer;
protected final String dictionaryFileName;
protected DictionaryIdLookup dictionaryIdLookup;
protected DictionaryWriter<T> dictionaryWriter;
@ -66,18 +65,15 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
public ScalarNestedCommonFormatColumnSerializer(
String name,
String dictionaryFileName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
Closer closer
)
{
this.name = name;
this.dictionaryFileName = dictionaryFileName;
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.indexSpec = indexSpec;
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 writeDictionaryFile(FileSmoosher smoosher) throws IOException;
@Override
public String getColumnName()
{
@ -220,7 +218,7 @@ public abstract class ScalarNestedCommonFormatColumnSerializer<T> extends Nested
}
writeV0Header(channel, columnNameBytes);
writeInternal(smoosher, dictionaryWriter, dictionaryFileName);
writeDictionaryFile(smoosher);
writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME);
writeValueColumn(smoosher);
writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME);

View File

@ -21,20 +21,17 @@ package org.apache.druid.segment.nested;
import com.google.common.base.Supplier;
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.StringUtils;
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnConfig;
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.data.BitmapSerdeFactory;
import org.apache.druid.segment.data.ColumnarInts;
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.Indexed;
import org.apache.druid.segment.data.VByte;
@ -71,39 +68,11 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
);
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) {
dictionarySupplier = 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
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;
}
dictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
mapper,
stringDictionaryBuffer,
byteOrder
);
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,

View File

@ -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.io.Closer;
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.ExpressionType;
import org.apache.druid.segment.IndexSpec;
@ -46,7 +47,7 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
Closer closer
)
{
super(name, STRING_DICTIONARY_FILE_NAME, indexSpec, segmentWriteOutMedium, closer);
super(name, indexSpec, segmentWriteOutMedium, closer);
}
@Override
@ -67,6 +68,15 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
name
);
dictionaryWriter.open();
dictionaryIdLookup = closer.register(
new DictionaryIdLookup(
name,
dictionaryWriter,
null,
null,
null
)
);
}
@Override
@ -89,7 +99,6 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
// null is always 0
dictionaryWriter.write(null);
dictionaryIdLookup.addString(null);
for (String value : strings) {
value = NullHandling.emptyToNullIfNeeded(value);
if (value == null) {
@ -97,7 +106,6 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
}
dictionaryWriter.write(value);
dictionaryIdLookup.addString(value);
}
dictionarySerialized = true;
}
@ -107,4 +115,17 @@ public class ScalarStringColumnSerializer extends ScalarNestedCommonFormatColumn
{
// 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);
}
}
}

View File

@ -23,7 +23,6 @@ import com.google.common.base.Supplier;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.collections.bitmap.ImmutableBitmap;
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.StringUtils;
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.ColumnIndexSupplier;
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.ValueType;
import org.apache.druid.segment.data.BitmapSerdeFactory;
import org.apache.druid.segment.data.ColumnarInts;
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.FrontCodedIndexed;
import org.apache.druid.segment.data.FrontCodedIntArrayIndexed;
import org.apache.druid.segment.data.GenericIndexed;
import org.apache.druid.segment.data.Indexed;
@ -92,8 +89,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
if (version == NestedCommonFormatColumnSerializer.V0) {
try {
final SmooshedFileMapper mapper = columnBuilder.getFileMapper();
final GenericIndexed<ByteBuffer> stringDictionary;
final Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier;
final Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier;
final Supplier<FixedIndexed<Long>> longDictionarySupplier;
final Supplier<FixedIndexed<Double>> doubleDictionarySupplier;
final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
@ -105,34 +101,11 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME
);
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) {
frontCodedStringDictionarySupplier = FrontCodedIndexed.read(
stringDictionaryBuffer,
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;
}
stringDictionarySupplier = StringEncodingStrategies.getStringDictionarySupplier(
mapper,
stringDictionaryBuffer,
byteOrder
);
final ByteBuffer encodedValueColumn = NestedCommonFormatColumnPartSerde.loadInternalFile(
mapper,
columnName,
@ -213,8 +186,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
return new VariantColumnAndIndexSupplier(
logicalType,
variantTypeByte,
stringDictionary,
frontCodedStringDictionarySupplier,
stringDictionarySupplier,
longDictionarySupplier,
doubleDictionarySupplier,
arrayDictionarySupplier,
@ -255,8 +227,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
public VariantColumnAndIndexSupplier(
ColumnType logicalType,
@Nullable Byte variantTypeSetByte,
GenericIndexed<ByteBuffer> stringDictionary,
Supplier<FrontCodedIndexed> frontCodedStringDictionarySupplier,
Supplier<? extends Indexed<ByteBuffer>> stringDictionarySupplier,
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<FixedIndexed<Double>> doubleDictionarySupplier,
Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier,
@ -271,9 +242,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
{
this.logicalType = logicalType;
this.variantTypeSetByte = variantTypeSetByte;
stringDictionarySupplier = frontCodedStringDictionarySupplier != null
? frontCodedStringDictionarySupplier
: stringDictionary::singleThreaded;
this.stringDictionarySupplier = stringDictionarySupplier;
this.longDictionarySupplier = longDictionarySupplier;
this.doubleDictionarySupplier = doubleDictionarySupplier;
this.arrayDictionarySupplier = arrayDictionarySupplier;

View File

@ -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.io.Closer;
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.math.expr.ExprEval;
import org.apache.druid.segment.ColumnValueSelector;
@ -94,7 +95,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
this.segmentWriteOutMedium = segmentWriteOutMedium;
this.indexSpec = indexSpec;
this.closer = closer;
this.dictionaryIdLookup = new DictionaryIdLookup();
}
@Override
@ -151,6 +151,15 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
arrayDictionaryWriter.open();
arrayElementDictionaryWriter = new FixedIndexedIntWriter(segmentWriteOutMedium, true);
arrayElementDictionaryWriter.open();
dictionaryIdLookup = closer.register(
new DictionaryIdLookup(
name,
dictionaryWriter,
longDictionaryWriter,
doubleDictionaryWriter,
arrayDictionaryWriter
)
);
}
@Override
@ -177,7 +186,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
// null is always 0
dictionaryWriter.write(null);
dictionaryIdLookup.addString(null);
for (String value : strings) {
value = NullHandling.emptyToNullIfNeeded(value);
if (value == null) {
@ -185,7 +193,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
}
dictionaryWriter.write(value);
dictionaryIdLookup.addString(value);
}
for (Long value : longs) {
@ -193,7 +200,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
continue;
}
longDictionaryWriter.write(value);
dictionaryIdLookup.addLong(value);
}
for (Double value : doubles) {
@ -201,7 +207,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
continue;
}
doubleDictionaryWriter.write(value);
dictionaryIdLookup.addDouble(value);
}
for (int[] value : arrays) {
@ -209,7 +214,6 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
continue;
}
arrayDictionaryWriter.write(value);
dictionaryIdLookup.addArray(value);
}
dictionarySerialized = true;
}
@ -378,10 +382,30 @@ public class VariantColumnSerializer extends NestedCommonFormatColumnSerializer
channel.write(ByteBuffer.wrap(new byte[]{variantTypeSetByte}));
}
writeInternal(smoosher, dictionaryWriter, STRING_DICTIONARY_FILE_NAME);
writeInternal(smoosher, longDictionaryWriter, LONG_DICTIONARY_FILE_NAME);
writeInternal(smoosher, doubleDictionaryWriter, DOUBLE_DICTIONARY_FILE_NAME);
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
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);
}
if (dictionaryIdLookup.getLongBuffer() != null) {
writeInternal(smoosher, dictionaryIdLookup.getLongBuffer(), LONG_DICTIONARY_FILE_NAME);
} else {
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);
}
if (dictionaryIdLookup.getArrayBuffer() != null) {
writeInternal(smoosher, dictionaryIdLookup.getArrayBuffer(), ARRAY_DICTIONARY_FILE_NAME);
} else {
writeInternal(smoosher, arrayDictionaryWriter, ARRAY_DICTIONARY_FILE_NAME);
}
writeInternal(smoosher, arrayElementDictionaryWriter, ARRAY_ELEMENT_DICTIONARY_FILE_NAME);
writeInternal(smoosher, encodedValueSerializer, ENCODED_VALUE_COLUMN_FILE_NAME);
writeInternal(smoosher, bitmapIndexWriter, BITMAP_INDEX_FILE_NAME);

View File

@ -28,11 +28,10 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap;
import org.apache.druid.collections.spatial.ImmutableRTree;
import org.apache.druid.io.Channels;
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.segment.column.ColumnBuilder;
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.data.BitmapSerde;
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.CompressedVSizeColumnarMultiIntsSupplier;
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.GenericIndexedWriter;
import org.apache.druid.segment.data.ImmutableRTreeObjectStrategy;
@ -312,37 +309,12 @@ public class DictionaryEncodedColumnPartSerde implements ColumnPartSerde
builder.setType(ValueType.STRING);
final int dictionaryStartPosition = buffer.position();
final byte dictionaryVersion = buffer.get();
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier;
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(
final Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier =
StringEncodingStrategies.getStringDictionarySupplier(
builder.getFileMapper(),
buffer,
GenericIndexed.UTF8_STRATEGY,
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;
}
byteOrder
);
final WritableSupplier<ColumnarInts> rSingleValuedColumn;
final WritableSupplier<ColumnarMultiInts> rMultiValuedColumn;

View File

@ -59,6 +59,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.incremental.IndexSizeExceededException;
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import org.junit.After;
@ -81,7 +82,7 @@ import java.util.stream.IntStream;
import java.util.stream.Stream;
@RunWith(Parameterized.class)
public class DoubleStorageTest
public class DoubleStorageTest extends InitializedNullHandlingTest
{
private static final SegmentMetadataQueryRunnerFactory METADATA_QR_FACTORY = new SegmentMetadataQueryRunnerFactory(

View File

@ -141,6 +141,35 @@ public class FixedIndexedTest extends InitializedNullHandlingTest
for (Long aLong : LONGS) {
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()
{
@Override