Skip to content

Commit

Permalink
Improve String Last/First Storage Efficiency
Browse files Browse the repository at this point in the history
-Add classes for writing cell values in LZ4 block compressed format.
Payloads are indexed by element number for efficient random lookup
-update SerializablePairLongStringComplexMetricSerde to use block
compression
-SerializablePairLongStringComplexMetricSerde also uses delta encoding
of the Long by doing 2-pass encoding: buffers first to find min/max
numbers and delta-encodes as integers if possible

Entry points for doing block-compressed storage of byte[] payloads
are the CellWriter and CellReader class. See
SerializablePairLongStringComplexMetricSerde for how these are used
along with how to do full column-based storage (delta encoding here)
which includes 2-pass encoding to compute a column header
  • Loading branch information
rash67 committed Aug 10, 2022
1 parent b4985cc commit bd68d14
Show file tree
Hide file tree
Showing 51 changed files with 4,636 additions and 50 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@
import org.apache.druid.query.aggregation.LongMinAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.aggregation.SerializablePairLongStringSerde;
import org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerde;
import org.apache.druid.query.aggregation.any.DoubleAnyAggregatorFactory;
import org.apache.druid.query.aggregation.any.FloatAnyAggregatorFactory;
import org.apache.druid.query.aggregation.any.LongAnyAggregatorFactory;
Expand Down Expand Up @@ -81,7 +81,7 @@ public AggregatorsModule()

ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde());
ComplexMetrics.registerSerde("preComputedHyperUnique", new PreComputedHyperUniquesSerde());
ComplexMetrics.registerSerde("serializablePairLongString", new SerializablePairLongStringSerde());
ComplexMetrics.registerSerde("serializablePairLongString", new SerializablePairLongStringComplexMetricSerde());

setMixInAnnotation(AggregatorFactory.class, AggregatorFactoryMixin.class);
setMixInAnnotation(PostAggregator.class, PostAggregatorMixin.class);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,174 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.druid.query.aggregation;

import org.apache.druid.java.util.common.io.smoosh.FileSmoosher;
import org.apache.druid.segment.serde.Serializer;
import org.apache.druid.segment.serde.cell.CellWriter;
import org.apache.druid.segment.serde.cell.DeserializingIOIterator;
import org.apache.druid.segment.serde.cell.IOIterator;
import org.apache.druid.segment.serde.cell.IntSerializer;
import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.apache.druid.segment.writeout.WriteOutBytes;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import java.io.IOException;
import java.nio.channels.WritableByteChannel;

public class SerializablePairLongStringBufferStore
{
private static final SerializablePairLongStringSimpleStagedSerde SERDE =
new SerializablePairLongStringSimpleStagedSerde();

private final WriteOutBytes writeOutBytes;
private final IntSerializer intSerializer = new IntSerializer();

private long minValue = Long.MAX_VALUE;
private long maxValue = Long.MIN_VALUE;

public SerializablePairLongStringBufferStore(WriteOutBytes writeOutBytes)
{
this.writeOutBytes = writeOutBytes;
}

public void store(@Nullable SerializablePairLongString pairLongString) throws IOException
{
if (pairLongString != null && pairLongString.lhs != null) {
minValue = Math.min(minValue, pairLongString.lhs);
maxValue = Math.max(maxValue, pairLongString.lhs);
}

byte[] bytes = SERDE.serialize(pairLongString);

writeOutBytes.write(intSerializer.serialize(bytes.length));
writeOutBytes.write(bytes);
}

/**
* each call transfers the temporary buffer into an encoded, block-compessed buffer of the segment. It is ready to be
* transferred to a {@link WritableByteChannel}
*
* @param byteBufferProvider - provides a ByteBuffer used for block compressed encoding
* @param segmentWriteOutMedium - used to create temporary storage
* @return encoded buffer ready to be stored
* @throws IOException
*/
public TransferredBuffer transferToRowWriter(
NativeClearedByteBufferProvider byteBufferProvider,
SegmentWriteOutMedium segmentWriteOutMedium
) throws IOException
{
SerializablePairLongStringColumnHeader columnHeader = createColumnHeader();
SerializablePairLongStringDeltaEncodedStagedSerde serde =
new SerializablePairLongStringDeltaEncodedStagedSerde(
columnHeader.getMinValue(),
columnHeader.isUseIntegerDeltas()
);

// try-with-resources will call cellWriter.close() an extra time in the normal case, but it protects against
// buffer leaking in the case of an exception. In the normal path, close() performs some finalization of
// the CellWriter object. We want that object state finalized before creating the TransferredBuffer as a point of
// good style (though strictly speaking, it works fine to pass it in before calling close since TransferredBuffer
// does not do anything in the constructor with the object)
try (CellWriter cellWriter = new CellWriter.Builder(byteBufferProvider, segmentWriteOutMedium).build()) {
IOIterator<SerializablePairLongString> bufferIterator = new DeserializingIOIterator<>(
writeOutBytes.asInputStream(),
SERDE
);

while (bufferIterator.hasNext()) {
SerializablePairLongString pairLongString = bufferIterator.next();
byte[] serialized = serde.serialize(pairLongString);

cellWriter.write(serialized);
}

cellWriter.close();

return new TransferredBuffer(cellWriter, columnHeader);
}
}

@Nonnull
public SerializablePairLongStringColumnHeader createColumnHeader()
{
long maxDelta = maxValue - minValue;
SerializablePairLongStringColumnHeader columnHeader;

if (minValue < maxValue && maxDelta < 0 || minValue > maxValue) {
// true iff
// 1. we have overflow in our range || 2. we have only seen null values
// in this case, effectively disable delta encoding by using longs and a min value of 0
maxDelta = Long.MAX_VALUE;
minValue = 0;
}

if (maxDelta <= Integer.MAX_VALUE) {
columnHeader = new SerializablePairLongStringColumnHeader(
SerializablePairLongStringComplexMetricSerde.EXPECTED_VERSION,
true,
minValue
);
} else {
columnHeader = new SerializablePairLongStringColumnHeader(
SerializablePairLongStringComplexMetricSerde.EXPECTED_VERSION,
false,
minValue
);
}
return columnHeader;
}

public IOIterator<SerializablePairLongString> iterator() throws IOException
{
return new DeserializingIOIterator<>(writeOutBytes.asInputStream(), SERDE);
}

/**
* contains serialized data that is compressed and delta-encoded (Long)
* It's ready to be transferred to a {@link WritableByteChannel}
*/
public static class TransferredBuffer implements Serializer
{
private final CellWriter cellWriter;
private final SerializablePairLongStringColumnHeader columnHeader;

public TransferredBuffer(CellWriter cellWriter, SerializablePairLongStringColumnHeader columnHeader)
{
this.cellWriter = cellWriter;
this.columnHeader = columnHeader;
}

@Override
public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException
{
columnHeader.transferTo(channel);
cellWriter.writeTo(channel, smoosher);
}

@Override
public long getSerializedSize()
{
return columnHeader.getSerializedSize() + cellWriter.getSerializedSize();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.druid.query.aggregation;

import com.google.common.base.Objects;
import org.apache.druid.segment.serde.cell.LongSerializer;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.WritableByteChannel;

public class SerializablePairLongStringColumnHeader
{
// header size is 4 bytes for word alignment for LZ4 (minmatch) compression
private static final int HEADER_SIZE_BYTES = 4;
private static final int USE_INTEGER_MASK = 0x80;
private static final int VERSION_INDEX = 0;
private static final int ENCODING_INDEX = 1;

private final LongSerializer longSerializer = new LongSerializer();
private final byte[] bytes;
private final long minValue;

private SerializablePairLongStringColumnHeader(byte[] bytes, long minTimestamp)
{
this.bytes = bytes;
this.minValue = minTimestamp;
}

public SerializablePairLongStringColumnHeader(byte version, boolean useIntegerDeltas, long minTimestamp)
{
this.minValue = minTimestamp;
bytes = new byte[HEADER_SIZE_BYTES];
bytes[VERSION_INDEX] = version;

if (useIntegerDeltas) {
bytes[ENCODING_INDEX] |= USE_INTEGER_MASK;
}
}

public static SerializablePairLongStringColumnHeader fromBuffer(ByteBuffer byteBuffer)
{
byte[] bytes = new byte[HEADER_SIZE_BYTES];

byteBuffer.get(bytes);

long minTimestamp = byteBuffer.getLong();

return new SerializablePairLongStringColumnHeader(bytes, minTimestamp);
}

public SerializablePairLongStringDeltaEncodedStagedSerde createSerde()
{
return new SerializablePairLongStringDeltaEncodedStagedSerde(minValue, isUseIntegerDeltas());
}

public void transferTo(WritableByteChannel channel) throws IOException
{
channel.write(ByteBuffer.wrap(bytes));
channel.write(longSerializer.serialize(minValue));
}

public byte getVersion()
{
return bytes[VERSION_INDEX];
}

public boolean isUseIntegerDeltas()
{
return (bytes[ENCODING_INDEX] & USE_INTEGER_MASK) != 0;
}

public long getMinValue()
{
return minValue;
}

public int getSerializedSize()
{
return HEADER_SIZE_BYTES + Long.BYTES;
}

@Override
public String toString()
{
return Objects.toStringHelper(this)
.add("bytes", bytes)
.add("minValue", minValue)
.toString();
}
}
Loading

0 comments on commit bd68d14

Please sign in to comment.