From bd68d14ec71e96f5d2053173da0913767c7ec2e8 Mon Sep 17 00:00:00 2001 From: Sam Rash Date: Tue, 9 Aug 2022 13:05:08 -0700 Subject: [PATCH] Improve String Last/First Storage Efficiency -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 --- .../druid/jackson/AggregatorsModule.java | 4 +- ...SerializablePairLongStringBufferStore.java | 174 ++++++++ ...erializablePairLongStringColumnHeader.java | 108 +++++ ...lizablePairLongStringColumnSerializer.java | 117 ++++++ ...rializablePairLongStringComplexColumn.java | 134 ++++++ ...ablePairLongStringComplexMetricSerde.java} | 77 ++-- ...PairLongStringDeltaEncodedStagedSerde.java | 131 ++++++ ...izablePairLongStringSimpleStagedSerde.java | 102 +++++ .../cell/BlockCompressedPayloadBuffer.java | 123 ++++++ .../BlockCompressedPayloadBufferFactory.java | 65 +++ .../cell/BlockCompressedPayloadReader.java | 155 +++++++ .../BlockCompressedPayloadSerializer.java | 57 +++ .../cell/BlockCompressedPayloadWriter.java | 118 ++++++ .../segment/serde/cell/BlockIndexWriter.java | 30 ++ .../segment/serde/cell/CellIndexReader.java | 46 +++ .../segment/serde/cell/CellIndexWriter.java | 73 ++++ .../druid/segment/serde/cell/CellReader.java | 94 +++++ .../druid/segment/serde/cell/CellWriter.java | 191 +++++++++ .../serde/cell/DeserializingIOIterator.java | 108 +++++ .../druid/segment/serde/cell/IOIterator.java | 29 ++ .../druid/segment/serde/cell/IndexWriter.java | 80 ++++ .../segment/serde/cell/IntIndexView.java | 69 ++++ .../segment/serde/cell/IntSerializer.java | 45 +++ .../segment/serde/cell/LongSerializer.java | 43 ++ .../cell/NativeClearedByteBufferProvider.java | 45 +++ .../segment/serde/cell/NumberSerializer.java | 29 ++ .../segment/serde/cell/PayloadEntrySpan.java | 42 ++ .../druid/segment/serde/cell/StagedSerde.java | 64 +++ .../segment/serde/cell/StorableBuffer.java | 48 +++ ...alizablePairLongStringBufferStoreTest.java | 382 ++++++++++++++++++ ...ePairLongStringComplexMetricSerdeTest.java | 237 +++++++++++ ...LongStringDeltaEncodedStagedSerdeTest.java | 99 +++++ ...lePairLongStringSimpleStagedSerdeTest.java | 70 ++++ .../SingleValueColumnValueSelector.java | 79 ++++ .../first/StringFirstTimeseriesQueryTest.java | 4 +- .../last/StringLastTimeseriesQueryTest.java | 4 +- .../segment/serde/ComplexMetricsTest.java | 6 +- .../BlockCompressedPayloadWriterTest.java | 56 +++ ...kCompressedPayloadWriterToBytesWriter.java | 90 +++++ .../serde/cell/ByteWriterTestHelper.java | 372 +++++++++++++++++ .../serde/cell/BytesReadWriteTest.java | 47 +++ .../serde/cell/BytesReadWriteTestBase.java | 244 +++++++++++ .../serde/cell/BytesReadWriteTestCases.java | 28 ++ .../druid/segment/serde/cell/BytesWriter.java | 43 ++ .../serde/cell/BytesWriterBuilder.java | 36 ++ .../segment/serde/cell/CellWriterTest.java | 52 +++ .../serde/cell/CellWriterToBytesWriter.java | 90 +++++ .../segment/serde/cell/CommitTestSuite.java | 44 ++ .../segment/serde/cell/RandomStringUtils.java | 49 +++ .../segment/serde/cell/TestCaseResult.java | 54 +++ .../segment/serde/cell/TestCasesConfig.java | 199 +++++++++ 51 files changed, 4636 insertions(+), 50 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStore.java create mode 100644 processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnHeader.java create mode 100644 processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexColumn.java rename processing/src/main/java/org/apache/druid/query/aggregation/{SerializablePairLongStringSerde.java => SerializablePairLongStringComplexMetricSerde.java} (61%) create mode 100644 processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerde.java create mode 100644 processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerde.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBuffer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBufferFactory.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadReader.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/BlockIndexWriter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexReader.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexWriter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/CellReader.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/CellWriter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/DeserializingIOIterator.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/IndexWriter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/IntIndexView.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/IntSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/LongSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/NativeClearedByteBufferProvider.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/NumberSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/PayloadEntrySpan.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/StagedSerde.java create mode 100644 processing/src/main/java/org/apache/druid/segment/serde/cell/StorableBuffer.java create mode 100644 processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStoreTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerdeTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerdeTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/aggregation/SingleValueColumnValueSelector.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterToBytesWriter.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/ByteWriterTestHelper.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestBase.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestCases.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriter.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriterBuilder.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterToBytesWriter.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/CommitTestSuite.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/RandomStringUtils.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/TestCaseResult.java create mode 100644 processing/src/test/java/org/apache/druid/segment/serde/cell/TestCasesConfig.java diff --git a/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java b/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java index 155b8e7e7b7a..9f7927305065 100644 --- a/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java +++ b/processing/src/main/java/org/apache/druid/jackson/AggregatorsModule.java @@ -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; @@ -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); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStore.java b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStore.java new file mode 100644 index 000000000000..103faceda62f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStore.java @@ -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 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 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(); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnHeader.java b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnHeader.java new file mode 100644 index 000000000000..63b46e14e21f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnHeader.java @@ -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(); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnSerializer.java b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnSerializer.java new file mode 100644 index 000000000000..d41bea3fbf93 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringColumnSerializer.java @@ -0,0 +1,117 @@ +/* + * 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.Preconditions; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.GenericColumnSerializer; +import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +/** + * valid call sequence + * + * open()+serialize()*(getSerializedSize()|writeTo())* + * + * getSerializedSize() / writeTo() effectively function as a close call, but each may be called multiple times and has + * no effect on one another. + */ +@SuppressWarnings("NotNullFieldNotInitialized") +public class SerializablePairLongStringColumnSerializer implements GenericColumnSerializer +{ + private final SegmentWriteOutMedium segmentWriteOutMedium; + private final NativeClearedByteBufferProvider byteBufferProvider; + + private State state = State.START; + private SerializablePairLongStringBufferStore bufferStore; + private SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer; + + public SerializablePairLongStringColumnSerializer( + SegmentWriteOutMedium segmentWriteOutMedium, + NativeClearedByteBufferProvider byteBufferProvider + ) + { + this.segmentWriteOutMedium = segmentWriteOutMedium; + this.byteBufferProvider = byteBufferProvider; + } + + @Override + public void open() throws IOException + { + Preconditions.checkState(state == State.START || state == State.OPEN, "open called in invalid state %s", state); + + if (state == State.START) { + bufferStore = new SerializablePairLongStringBufferStore(segmentWriteOutMedium.makeWriteOutBytes()); + state = State.OPEN; + } + } + + @Override + public void serialize(ColumnValueSelector selector) throws IOException + { + Preconditions.checkState(state == State.OPEN, "serialize called in invalid state %s", state); + + SerializablePairLongString pairLongString = selector.getObject(); + + bufferStore.store(pairLongString); + } + + @Override + public long getSerializedSize() throws IOException + { + Preconditions.checkState( + state != State.START, + "getSerializedSize called in invalid state %s (must have opened at least)", + state + ); + + transferToRowWriterIfNecessary(); + + return transferredBuffer.getSerializedSize(); + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + Preconditions.checkState(state != State.START, "writeTo called in invalid state %s", state); + transferToRowWriterIfNecessary(); + transferredBuffer.writeTo(channel, smoosher); + } + + private void transferToRowWriterIfNecessary() throws IOException + { + if (state == State.OPEN) { + transferredBuffer = bufferStore.transferToRowWriter(byteBufferProvider, segmentWriteOutMedium); + state = State.CLOSED; + } + } + + + private enum State + { + START, + OPEN, + CLOSED, + } +} diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexColumn.java b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexColumn.java new file mode 100644 index 000000000000..b628961ad319 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexColumn.java @@ -0,0 +1,134 @@ +/* + * 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.Preconditions; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.column.ComplexColumn; +import org.apache.druid.segment.serde.cell.CellReader; +import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class SerializablePairLongStringComplexColumn implements ComplexColumn +{ + private final Closer closer; + private final int serializedSize; + private final CellReader cellReader; + private final SerializablePairLongStringDeltaEncodedStagedSerde serde; + + public SerializablePairLongStringComplexColumn( + CellReader cellReader, + SerializablePairLongStringDeltaEncodedStagedSerde serde, + Closer closer, + int serializedSize + ) + { + this.cellReader = cellReader; + this.serde = serde; + this.closer = closer; + this.serializedSize = serializedSize; + } + + @Override + public Class getClazz() + { + return SerializablePairLongString.class; + } + + @Override + public String getTypeName() + { + return SerializablePairLongStringComplexMetricSerde.TYPE_NAME; + } + + @SuppressWarnings("ConstantConditions") + @Override + public Object getRowValue(int rowNum) + { + // nulls are handled properly by the aggregator + return serde.deserialize(cellReader.getCell(rowNum)); + } + + @Override + public int getLength() + { + return serializedSize; + } + + @Override + public void close() + { + try { + closer.close(); + } + catch (IOException e) { + throw new RE(e, "error closing " + getClass().getName()); + } + } + + public static class Factory + { + private final int serializedSize; + private final NativeClearedByteBufferProvider byteBufferProvider; + private final SerializablePairLongStringDeltaEncodedStagedSerde serde; + private final CellReader.Builder cellReaderBuilder; + + public Factory(ByteBuffer buffer, NativeClearedByteBufferProvider byteBufferProvider) + { + this.byteBufferProvider = byteBufferProvider; + + ByteBuffer masterByteBuffer = buffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder()); + + serializedSize = masterByteBuffer.remaining(); + + SerializablePairLongStringColumnHeader columnHeader = + SerializablePairLongStringColumnHeader.fromBuffer(masterByteBuffer); + + Preconditions.checkArgument( + columnHeader.getVersion() == SerializablePairLongStringComplexMetricSerde.EXPECTED_VERSION, + "version %s expected, got %s", + SerializablePairLongStringComplexMetricSerde.EXPECTED_VERSION, + columnHeader.getVersion() + ); + serde = columnHeader.createSerde(); + cellReaderBuilder = new CellReader.Builder(masterByteBuffer); + } + + public SerializablePairLongStringComplexColumn create() + { + Closer closer = Closer.create(); + ResourceHolder cellIndexUncompressedBlockHolder = byteBufferProvider.get(); + ResourceHolder dataUncompressedBlockHolder = byteBufferProvider.get(); + + closer.register(cellIndexUncompressedBlockHolder); + closer.register(dataUncompressedBlockHolder); + + CellReader cellReader = + cellReaderBuilder.build(cellIndexUncompressedBlockHolder.get(), dataUncompressedBlockHolder.get()); + + return new SerializablePairLongStringComplexColumn(cellReader, serde, closer, serializedSize); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringSerde.java b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerde.java similarity index 61% rename from processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringSerde.java rename to processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerde.java index 49300ff53176..20e40aa8a37b 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringSerde.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerde.java @@ -21,7 +21,6 @@ import org.apache.druid.collections.SerializablePair; import org.apache.druid.data.input.InputRow; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.GenericColumnSerializer; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.data.GenericIndexed; @@ -29,7 +28,7 @@ import org.apache.druid.segment.serde.ComplexColumnPartSupplier; import org.apache.druid.segment.serde.ComplexMetricExtractor; import org.apache.druid.segment.serde.ComplexMetricSerde; -import org.apache.druid.segment.serde.LargeColumnSupportedComplexColumnSerializer; +import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; import javax.annotation.Nullable; @@ -39,14 +38,19 @@ /** * The SerializablePairLongStringSerde serializes a Long-String pair (SerializablePairLongString). * The serialization structure is: Long:Integer:String + * The Long is delta-encoded for the column in order to potentially reduce the size to an integer so it may be stored + * as: Integer:Integer:String + * + * Future work: dictionary encoding of the String may be performed *

* The class is used on first/last String aggregators to store the time and the first/last string. - * Long:Integer:String -> Timestamp:StringSize:StringData + * [Integer|Long]:Integer:String -> delta:StringSize:StringData --(delta decoded)--> TimeStamp:StringSize:StringData + * (see {@link SerializablePairLongStringDeltaEncodedStagedSerde )} */ -public class SerializablePairLongStringSerde extends ComplexMetricSerde +public class SerializablePairLongStringComplexMetricSerde extends ComplexMetricSerde { - - private static final String TYPE_NAME = "serializablePairLongString"; + public static final byte EXPECTED_VERSION = 3; + public static final String TYPE_NAME = "serializablePairLongString"; // Null SerializablePairLongString values are put first private static final Comparator COMPARATOR = Comparator.nullsFirst( // assumes that the LHS of the pair will never be null @@ -54,6 +58,9 @@ public class SerializablePairLongStringSerde extends ComplexMetricSerde .thenComparing(SerializablePair::getRhs, Comparator.nullsFirst(Comparator.naturalOrder())) ); + private static final SerializablePairLongStringSimpleStagedSerde SERDE = + new SerializablePairLongStringSimpleStagedSerde(); + @Override public String getTypeName() { @@ -61,9 +68,9 @@ public String getTypeName() } @Override - public ComplexMetricExtractor getExtractor() + public ComplexMetricExtractor getExtractor() { - return new ComplexMetricExtractor() + return new ComplexMetricExtractor() { @Override public Class extractedClass() @@ -82,12 +89,20 @@ public Object extractValue(InputRow inputRow, String metricName) @Override public void deserializeColumn(ByteBuffer buffer, ColumnBuilder columnBuilder) { - final GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper()); - columnBuilder.setComplexColumnSupplier(new ComplexColumnPartSupplier(getTypeName(), column)); + byte version = buffer.get(buffer.position()); + + if (version == 0 || version == 1 || version == 2) { + GenericIndexed column = GenericIndexed.read(buffer, getObjectStrategy(), columnBuilder.getFileMapper()); + columnBuilder.setComplexColumnSupplier(new ComplexColumnPartSupplier(getTypeName(), column)); + } else { + SerializablePairLongStringComplexColumn.Factory factory = + new SerializablePairLongStringComplexColumn.Factory(buffer, NativeClearedByteBufferProvider.INSTANCE); + columnBuilder.setComplexColumnSupplier(factory::create); + } } @Override - public ObjectStrategy getObjectStrategy() + public ObjectStrategy getObjectStrategy() { return new ObjectStrategy() { @@ -106,48 +121,28 @@ public Class getClazz() @Override public SerializablePairLongString fromByteBuffer(ByteBuffer buffer, int numBytes) { - final ByteBuffer readOnlyBuffer = buffer.asReadOnlyBuffer(); - - long lhs = readOnlyBuffer.getLong(); - int stringSize = readOnlyBuffer.getInt(); + ByteBuffer readOnlyByteBuffer = buffer.asReadOnlyBuffer().order(buffer.order()); - String lastString = null; - if (stringSize > 0) { - byte[] stringBytes = new byte[stringSize]; - readOnlyBuffer.get(stringBytes, 0, stringSize); - lastString = StringUtils.fromUtf8(stringBytes); - } + readOnlyByteBuffer.limit(buffer.position() + numBytes); - return new SerializablePairLongString(lhs, lastString); + return SERDE.deserialize(readOnlyByteBuffer); } + @SuppressWarnings("NullableProblems") @Override public byte[] toBytes(SerializablePairLongString val) { - String rhsString = val.rhs; - ByteBuffer bbuf; - - if (rhsString != null) { - byte[] rhsBytes = StringUtils.toUtf8(rhsString); - bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES + rhsBytes.length); - bbuf.putLong(val.lhs); - bbuf.putInt(Long.BYTES, rhsBytes.length); - bbuf.position(Long.BYTES + Integer.BYTES); - bbuf.put(rhsBytes); - } else { - bbuf = ByteBuffer.allocate(Long.BYTES + Integer.BYTES); - bbuf.putLong(val.lhs); - bbuf.putInt(Long.BYTES, 0); - } - - return bbuf.array(); + return SERDE.serialize(val); } }; } @Override - public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) + public GenericColumnSerializer getSerializer(SegmentWriteOutMedium segmentWriteOutMedium, String column) { - return LargeColumnSupportedComplexColumnSerializer.create(segmentWriteOutMedium, column, this.getObjectStrategy()); + return new SerializablePairLongStringColumnSerializer( + segmentWriteOutMedium, + NativeClearedByteBufferProvider.INSTANCE + ); } } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerde.java b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerde.java new file mode 100644 index 000000000000..b576dc09f08e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerde.java @@ -0,0 +1,131 @@ +/* + * 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.Preconditions; +import com.google.common.primitives.Ints; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.serde.cell.StagedSerde; +import org.apache.druid.segment.serde.cell.StorableBuffer; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * serializes a Long/String pair in the context of a column/segment. Uses the minValue to perform delta + * encoding/decoding and if the range of the segment fits in an integer (useIntegerDelta), the format is + * Integer:Integer:bytes + * + * otherwise + * Long:Integer:bytes + */ +public class SerializablePairLongStringDeltaEncodedStagedSerde implements StagedSerde +{ + private static final byte[] EMPTY_BYTES = new byte[0]; + + private final long minValue; + private final boolean useIntegerDelta; + + public SerializablePairLongStringDeltaEncodedStagedSerde(long minValue, boolean useIntegerDelta) + { + this.minValue = minValue; + this.useIntegerDelta = useIntegerDelta; + } + + @Override + public StorableBuffer serializeDelayed(@Nullable SerializablePairLongString value) + { + if (value == null) { + return StorableBuffer.EMPTY; + } + + String rhsString = value.rhs; + byte[] rhsBytes = stringToUtf8Bytes(rhsString); + + return new StorableBuffer() + { + @Override + public void store(ByteBuffer byteBuffer) + { + Preconditions.checkNotNull(value.lhs, "Long in SerializablePairLongString must be non-null"); + + long delta = value.lhs - minValue; + + Preconditions.checkState(delta >= 0 || delta == value.lhs); + + if (useIntegerDelta) { + byteBuffer.putInt(Ints.checkedCast(delta)); + } else { + byteBuffer.putLong(delta); + } + + byteBuffer.putInt(rhsBytes.length); + + if (rhsBytes.length > 0) { + byteBuffer.put(rhsBytes); + } + } + + @Override + public int getSerializedSize() + { + return (useIntegerDelta ? Integer.BYTES : Long.BYTES) + Integer.BYTES + rhsBytes.length; + } + }; + } + + @Nullable + @Override + public SerializablePairLongString deserialize(ByteBuffer byteBuffer) + { + if (byteBuffer.remaining() == 0) { + return null; + } + + ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder()); + long lhs; + + if (useIntegerDelta) { + lhs = readOnlyBuffer.getInt(); + } else { + lhs = readOnlyBuffer.getLong(); + } + + lhs += minValue; + + int stringSize = readOnlyBuffer.getInt(); + String lastString = null; + + if (stringSize > 0) { + byte[] stringBytes = new byte[stringSize]; + + readOnlyBuffer.get(stringBytes, 0, stringSize); + lastString = StringUtils.fromUtf8(stringBytes); + } + + return new SerializablePairLongString(lhs, lastString); + } + + private static byte[] stringToUtf8Bytes(@Nullable String value) + { + return value == null ? EMPTY_BYTES : StringUtils.toUtf8Nullable(value); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerde.java b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerde.java new file mode 100644 index 000000000000..b0b7fc9eb5ae --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerde.java @@ -0,0 +1,102 @@ +/* + * 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.Preconditions; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.serde.cell.StagedSerde; +import org.apache.druid.segment.serde.cell.StorableBuffer; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +/** + * serializes a Long/String pair as + * Long:Integer:bytes + *

+ * or + * Long:StringSize:StringData + */ +public class SerializablePairLongStringSimpleStagedSerde implements StagedSerde +{ + private static final byte[] EMPTY_BYTES = new byte[0]; + + @Override + public StorableBuffer serializeDelayed(@Nullable SerializablePairLongString value) + { + if (value == null) { + return StorableBuffer.EMPTY; + } + + String rhsString = value.rhs; + byte[] rhsBytes = stringToUtf8Bytes(rhsString); + + return new StorableBuffer() + { + @Override + public void store(ByteBuffer byteBuffer) + { + Preconditions.checkNotNull(value.lhs, "Long in SerializablePairLongString must be non-null"); + + byteBuffer.putLong(value.lhs); + byteBuffer.putInt(rhsBytes.length); + + if (rhsBytes.length > 0) { + byteBuffer.put(rhsBytes); + } + } + + @Override + public int getSerializedSize() + { + return Long.BYTES + Integer.BYTES + rhsBytes.length; + } + }; + } + + @Nullable + @Override + public SerializablePairLongString deserialize(ByteBuffer byteBuffer) + { + if (byteBuffer.remaining() == 0) { + return null; + } + + ByteBuffer readOnlyBuffer = byteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder()); + long lhs = readOnlyBuffer.getLong(); + int stringSize = readOnlyBuffer.getInt(); + String lastString = null; + + if (stringSize > 0) { + byte[] stringBytes = new byte[stringSize]; + + readOnlyBuffer.get(stringBytes, 0, stringSize); + lastString = StringUtils.fromUtf8(stringBytes); + } + + return new SerializablePairLongString(lhs, lastString); + } + + private static byte[] stringToUtf8Bytes(@Nullable String value) + { + return value == null ? EMPTY_BYTES : StringUtils.toUtf8(value); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBuffer.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBuffer.java new file mode 100644 index 000000000000..31bc231dd349 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBuffer.java @@ -0,0 +1,123 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class BlockCompressedPayloadBuffer implements Closeable +{ + private final ByteBuffer currentBlock; + private final ByteBuffer compressedByteBuffer; + private final BlockIndexWriter blockIndexWriter; + private final WriteOutBytes dataOutBytes; + private final Closer closer; + private final CompressionStrategy.Compressor compressor; + + private boolean open = true; + + public BlockCompressedPayloadBuffer( + ByteBuffer currentBlock, + ByteBuffer compressedByteBuffer, + BlockIndexWriter blockIndexWriter, + WriteOutBytes dataOutBytes, + Closer closer, + CompressionStrategy.Compressor compressor + ) + { + currentBlock.clear(); + compressedByteBuffer.clear(); + this.currentBlock = currentBlock; + this.compressedByteBuffer = compressedByteBuffer; + this.closer = closer; + this.blockIndexWriter = blockIndexWriter; + this.dataOutBytes = dataOutBytes; + this.compressor = compressor; + } + + public void write(byte[] payload) throws IOException + { + Preconditions.checkNotNull(payload); + write(ByteBuffer.wrap(payload).order(ByteOrder.nativeOrder())); + } + + public void write(@Nullable ByteBuffer masterPayload) throws IOException + { + Preconditions.checkNotNull(masterPayload); + Preconditions.checkState(open, "cannot write to closed BlockCompressedPayloadWriter"); + ByteBuffer payload = masterPayload.asReadOnlyBuffer().order(masterPayload.order()); + + while (payload.hasRemaining()) { + int writeSize = Math.min(payload.remaining(), currentBlock.remaining()); + + payload.limit(payload.position() + writeSize); + currentBlock.put(payload); + + if (!currentBlock.hasRemaining()) { + flush(); + } + + payload.limit(masterPayload.limit()); + } + } + + @Override + public void close() throws IOException + { + closer.close(); + } + + public BlockCompressedPayloadSerializer closeToSerializer() throws IOException + { + if (open) { + if (currentBlock.position() > 0) { + flush(); + } + + blockIndexWriter.close(); + closer.close(); + open = false; + } + + return new BlockCompressedPayloadSerializer(blockIndexWriter, dataOutBytes); + } + + private void flush() throws IOException + { + Preconditions.checkState(open, "flush() on closed BlockCompressedPayloadWriter"); + currentBlock.flip(); + + ByteBuffer actualCompressedByteBuffer = compressor.compress(currentBlock, compressedByteBuffer); + int compressedBlockSize = actualCompressedByteBuffer.limit(); + + blockIndexWriter.persistAndIncrement(compressedBlockSize); + dataOutBytes.write(actualCompressedByteBuffer); + currentBlock.clear(); + compressedByteBuffer.clear(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBufferFactory.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBufferFactory.java new file mode 100644 index 000000000000..1b74aa8c9a39 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadBufferFactory.java @@ -0,0 +1,65 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.IOException; +import java.nio.ByteBuffer; + +public class BlockCompressedPayloadBufferFactory +{ + private final NativeClearedByteBufferProvider byteBufferProvider; + private final SegmentWriteOutMedium writeOutMedium; + private final CompressionStrategy.Compressor compressor; + + public BlockCompressedPayloadBufferFactory( + NativeClearedByteBufferProvider byteBufferProvider, + SegmentWriteOutMedium writeOutMedium, + CompressionStrategy.Compressor compressor + ) + { + this.byteBufferProvider = byteBufferProvider; + this.writeOutMedium = writeOutMedium; + this.compressor = compressor; + } + + public BlockCompressedPayloadBuffer create() throws IOException + { + Closer closer = Closer.create(); + ResourceHolder currentBlockHolder = byteBufferProvider.get(); + + closer.register(currentBlockHolder); + + ByteBuffer compressedBlockByteBuffer = compressor.allocateOutBuffer(currentBlockHolder.get().capacity(), closer); + + return new BlockCompressedPayloadBuffer( + currentBlockHolder.get(), + compressedBlockByteBuffer, + new BlockIndexWriter(writeOutMedium.makeWriteOutBytes()), + writeOutMedium.makeWriteOutBytes(), + closer, + compressor + ); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadReader.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadReader.java new file mode 100644 index 000000000000..5588189c6416 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadReader.java @@ -0,0 +1,155 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.base.Preconditions; +import org.apache.druid.segment.data.CompressionStrategy; + +import javax.annotation.Nonnull; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class BlockCompressedPayloadReader +{ + private static final ByteBuffer NULL_CELL = ByteBuffer.wrap(new byte[0]); + private final IntIndexView blockIndexView; + private final ByteBuffer compressedBlocksByteBuffer; + private final ByteBuffer uncompressedByteBuffer; + private final int blockSize; + private final long maxValidUncompressedOffset; + private final CompressionStrategy.Decompressor decompressor; + + private int currentUncompressedBlockNumber = -1; + + private BlockCompressedPayloadReader( + IntIndexView blockIndexView, + ByteBuffer compressedBlocksByteBuffer, + ByteBuffer uncompressedByteBuffer, + CompressionStrategy.Decompressor decompressor + ) + { + this.blockIndexView = blockIndexView; + this.compressedBlocksByteBuffer = compressedBlocksByteBuffer; + this.uncompressedByteBuffer = uncompressedByteBuffer; + uncompressedByteBuffer.clear(); + blockSize = uncompressedByteBuffer.remaining(); + maxValidUncompressedOffset = Integer.MAX_VALUE * (long) blockSize; + this.decompressor = decompressor; + } + + /** + * + * @param originalByteBuffer - buffer as written byte {@link BlockCompressedPayloadWriter}. Not modified. + * @param uncompressedByteBuffer - should be native ordered ByteBuffer + * @param decompressor - decompressor for block compression + * @return + */ + public static BlockCompressedPayloadReader create( + ByteBuffer originalByteBuffer, + ByteBuffer uncompressedByteBuffer, + CompressionStrategy.Decompressor decompressor + ) + { + ByteBuffer masterByteBuffer = originalByteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder()); + + int blockIndexSize = masterByteBuffer.getInt(); + ByteBuffer blockIndexBuffer = masterByteBuffer.asReadOnlyBuffer().order(masterByteBuffer.order()); + blockIndexBuffer.limit(blockIndexBuffer.position() + blockIndexSize); + + masterByteBuffer.position(masterByteBuffer.position() + blockIndexSize); + + int dataStreamSize = masterByteBuffer.getInt(); + ByteBuffer compressedBlockStreamByteBuffer = masterByteBuffer.asReadOnlyBuffer().order(masterByteBuffer.order()); + compressedBlockStreamByteBuffer.limit(compressedBlockStreamByteBuffer.position() + dataStreamSize); + + return new BlockCompressedPayloadReader( + new IntIndexView(blockIndexBuffer), + compressedBlockStreamByteBuffer, + uncompressedByteBuffer, + decompressor + ); + } + + public ByteBuffer read(long uncompressedStart, int size) + { + if (size == 0) { + return NULL_CELL; + } + + Preconditions.checkArgument(uncompressedStart + size < maxValidUncompressedOffset); + + int blockNumber = (int) (uncompressedStart / blockSize); + int blockOffset = (int) (uncompressedStart % blockSize); + ByteBuffer currentUncompressedBlock = getUncompressedBlock(blockNumber); + + currentUncompressedBlock.position(blockOffset); + + if (size <= currentUncompressedBlock.remaining()) { + ByteBuffer resultByteBuffer = currentUncompressedBlock.asReadOnlyBuffer().order(ByteOrder.nativeOrder()); + + resultByteBuffer.limit(blockOffset + size); + + return resultByteBuffer; + } else { + byte[] payload = readMultiBlock(size, blockNumber, blockOffset); + + return ByteBuffer.wrap(payload).order(ByteOrder.nativeOrder()); + } + } + + @Nonnull + private byte[] readMultiBlock(int size, int blockNumber, int blockOffset) + { + byte[] payload = new byte[size]; + int bytesRead = 0; + + do { + ByteBuffer currentUncompressedBlock = getUncompressedBlock(blockNumber); + + currentUncompressedBlock.position(blockOffset); + + int readSizeBytes = Math.min(size - bytesRead, currentUncompressedBlock.remaining()); + + currentUncompressedBlock.get(payload, bytesRead, readSizeBytes); + bytesRead += readSizeBytes; + blockNumber++; + blockOffset = 0; + } while (bytesRead < size); + + return payload; + } + + private ByteBuffer getUncompressedBlock(int blockNumber) + { + if (currentUncompressedBlockNumber != blockNumber) { + IntIndexView.EntrySpan span = blockIndexView.getEntrySpan(blockNumber); + ByteBuffer compressedBlock = compressedBlocksByteBuffer.asReadOnlyBuffer() + .order(compressedBlocksByteBuffer.order()); + compressedBlock.position(compressedBlock.position() + span.getStart()); + compressedBlock.limit(compressedBlock.position() + span.getSize()); + uncompressedByteBuffer.clear(); + + decompressor.decompress(compressedBlock, span.getSize(), uncompressedByteBuffer); + currentUncompressedBlockNumber = blockNumber; + } + + return uncompressedByteBuffer; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadSerializer.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadSerializer.java new file mode 100644 index 000000000000..ec83c05d1f07 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadSerializer.java @@ -0,0 +1,57 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.primitives.Ints; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.serde.Serializer; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public class BlockCompressedPayloadSerializer implements Serializer +{ + private final IntSerializer intSerializer = new IntSerializer(); + private final BlockIndexWriter blockIndexWriter; + private final WriteOutBytes dataOutBytes; + + public BlockCompressedPayloadSerializer(BlockIndexWriter blockIndexWriter, WriteOutBytes dataOutBytes) + { + this.blockIndexWriter = blockIndexWriter; + this.dataOutBytes = dataOutBytes; + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + blockIndexWriter.transferTo(channel); + channel.write(intSerializer.serialize(dataOutBytes.size())); + dataOutBytes.writeTo(channel); + } + + @Override + public long getSerializedSize() + { + return blockIndexWriter.getSerializedSize() + + intSerializer.getSerializedSize() + + Ints.checkedCast(dataOutBytes.size()); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriter.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriter.java new file mode 100644 index 000000000000..563bd1e0c321 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriter.java @@ -0,0 +1,118 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.serde.Serializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +public class BlockCompressedPayloadWriter implements Serializer, Closeable +{ + private final BlockCompressedPayloadBuffer buffer; + private BlockCompressedPayloadSerializer serializer; + private State state = State.OPEN; + + private BlockCompressedPayloadWriter(BlockCompressedPayloadBuffer buffer) + { + this.buffer = buffer; + } + + public void write(byte[] payload) throws IOException + { + Preconditions.checkState(state == State.OPEN); + buffer.write(payload); + } + + public void write(ByteBuffer payload) throws IOException + { + Preconditions.checkState(state == State.OPEN); + buffer.write(payload); + } + + @Override + public void close() throws IOException + { + if (state == State.OPEN) { + serializer = buffer.closeToSerializer(); + state = State.CLOSED; + } + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + Preconditions.checkState(state == State.CLOSED); + serializer.writeTo(channel, smoosher); + } + + @Override + public long getSerializedSize() + { + Preconditions.checkState(state == State.CLOSED); + return serializer.getSerializedSize(); + } + + private enum State + { + OPEN, + CLOSED + } + + public static class Builder + { + private final NativeClearedByteBufferProvider byteBufferProvider; + private final SegmentWriteOutMedium writeOutMedium; + + private CompressionStrategy compressionStrategy = CompressionStrategy.LZ4; + + public Builder(NativeClearedByteBufferProvider byteBufferProvider, SegmentWriteOutMedium writeOutMedium) + { + this.byteBufferProvider = byteBufferProvider; + this.writeOutMedium = writeOutMedium; + } + + public Builder setCompressionStrategy(CompressionStrategy compressionStrategy) + { + this.compressionStrategy = compressionStrategy; + + return this; + } + + public BlockCompressedPayloadWriter build() throws IOException + { + BlockCompressedPayloadBufferFactory bufferFactory = new BlockCompressedPayloadBufferFactory( + byteBufferProvider, + writeOutMedium, + compressionStrategy.getCompressor() + ); + BlockCompressedPayloadBuffer payloadBuffer = bufferFactory.create(); + BlockCompressedPayloadWriter payloadWriter = new BlockCompressedPayloadWriter(payloadBuffer); + + return payloadWriter; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockIndexWriter.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockIndexWriter.java new file mode 100644 index 000000000000..986628690c4c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/BlockIndexWriter.java @@ -0,0 +1,30 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.segment.writeout.WriteOutBytes; + +public class BlockIndexWriter extends IndexWriter +{ + public BlockIndexWriter(WriteOutBytes outBytes) + { + super(outBytes, new IntSerializer()); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexReader.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexReader.java new file mode 100644 index 000000000000..11dc3fbbe493 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexReader.java @@ -0,0 +1,46 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.primitives.Ints; + +import javax.annotation.Nonnull; +import java.nio.ByteBuffer; + +public class CellIndexReader +{ + private final BlockCompressedPayloadReader payloadReader; + + public CellIndexReader(BlockCompressedPayloadReader payloadReader) + { + this.payloadReader = payloadReader; + } + + @Nonnull + public PayloadEntrySpan getEntrySpan(int entryNumber) + { + int position = entryNumber * Long.BYTES; + ByteBuffer payload = payloadReader.read(position, 2 * Long.BYTES); + long payloadValue = payload.getLong(); + long nextPayloadValue = payload.getLong(); + + return new PayloadEntrySpan(payloadValue, Ints.checkedCast(nextPayloadValue - payloadValue)); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexWriter.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexWriter.java new file mode 100644 index 000000000000..9e9f607f8534 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellIndexWriter.java @@ -0,0 +1,73 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.serde.Serializer; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + + +public class CellIndexWriter implements Serializer, Closeable +{ + private final LongSerializer longSerializer = new LongSerializer(); + private final BlockCompressedPayloadWriter payloadWriter; + + private long position = 0; + private boolean open = true; + + public CellIndexWriter(BlockCompressedPayloadWriter payloadWriter) + { + this.payloadWriter = payloadWriter; + } + + public void persistAndIncrement(int increment) throws IOException + { + Preconditions.checkArgument(increment >= 0); + Preconditions.checkState(open, "cannot write to closed CellIndex"); + payloadWriter.write(longSerializer.serialize(position)); + position += increment; + } + + @Override + public void close() throws IOException + { + if (open) { + payloadWriter.write(longSerializer.serialize(position)); + payloadWriter.close(); + open = false; + } + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + payloadWriter.writeTo(channel, smoosher); + } + + @Override + public long getSerializedSize() + { + return payloadWriter.getSerializedSize(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/CellReader.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellReader.java new file mode 100644 index 000000000000..2e5f48980205 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellReader.java @@ -0,0 +1,94 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.segment.data.CompressionStrategy; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class CellReader +{ + private final CellIndexReader cellIndexReader; + private final BlockCompressedPayloadReader dataReader; + + private CellReader(CellIndexReader cellIndexReader, BlockCompressedPayloadReader dataReader) + { + this.cellIndexReader = cellIndexReader; + this.dataReader = dataReader; + } + + public ByteBuffer getCell(int rowNumber) + { + PayloadEntrySpan payloadEntrySpan = cellIndexReader.getEntrySpan(rowNumber); + ByteBuffer payload = dataReader.read(payloadEntrySpan.getStart(), payloadEntrySpan.getSize()); + + return payload; + } + + public static class Builder + { + private final ByteBuffer cellIndexBuffer; + private final ByteBuffer dataStorageBuffer; + + private CompressionStrategy compressionStrategy = CompressionStrategy.LZ4; + + // this parses the buffer once into the index and data portions + public Builder(ByteBuffer originalByteBuffer) + { + ByteBuffer masterByteBuffer = originalByteBuffer.asReadOnlyBuffer().order(ByteOrder.nativeOrder()); + + int cellIndexSize = masterByteBuffer.getInt(); + cellIndexBuffer = masterByteBuffer.asReadOnlyBuffer().order(masterByteBuffer.order()); + cellIndexBuffer.limit(cellIndexBuffer.position() + cellIndexSize); + + masterByteBuffer.position(masterByteBuffer.position() + cellIndexSize); + + int dataStorageSize = masterByteBuffer.getInt(); + dataStorageBuffer = masterByteBuffer.asReadOnlyBuffer().order(masterByteBuffer.order()); + dataStorageBuffer.limit(dataStorageBuffer.position() + dataStorageSize); + } + + public Builder setCompressionStrategy(CompressionStrategy compressionStrategy) + { + this.compressionStrategy = compressionStrategy; + + return this; + } + + // this creates read only copies of the parsed ByteBuffers along with buffers for decompressing blocks into + public CellReader build(ByteBuffer cellIndexUncompressedByteBuffer, ByteBuffer dataUncompressedByteBuffer) + { + CellIndexReader cellIndexReader = new CellIndexReader(BlockCompressedPayloadReader.create( + cellIndexBuffer, + cellIndexUncompressedByteBuffer, + compressionStrategy.getDecompressor() + )); + BlockCompressedPayloadReader dataReader = BlockCompressedPayloadReader.create( + dataStorageBuffer, + dataUncompressedByteBuffer, + compressionStrategy.getDecompressor() + ); + CellReader cellReader = new CellReader(cellIndexReader, dataReader); + + return cellReader; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/CellWriter.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellWriter.java new file mode 100644 index 000000000000..1275efed5c4b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/CellWriter.java @@ -0,0 +1,191 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.serde.Serializer; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** + *

+ * serialized data is of the form:
+ *
+ *    
+ *    
+ *
+ * each of these items is stored in compressed streams of blocks with a block index.
+ *
+ * A BlockCompressedPayloadWriter stores byte[] payloads. These may be accessed by creating a
+ * BlockCompressedPayloadReader over the produced ByteBuffer. Reads may be done by giving a location in the
+ * uncompressed stream and a size
+ *
+ * NOTE: {@link BlockCompressedPayloadBuffer} does not store nulls on write(). However, the cellIndex stores an entry
+ * with a size of 0 for nulls and {@link CellReader} will return null for any null written
+ *
+ * * blockIndexSize:int
+ * |||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
+ * |      block index
+ * |      compressed block # -> block start in compressed stream position (relative to data start)
+ * |
+ * |      0: [block position: int]
+ * |      1: [block position: int]
+ * |      ...
+ * |      i: [block position: int]
+ * |      ...
+ * |      n: [block position: int]
+ * |      n+1: [total compressed size ] // stored to simplify invariant of
+ * |||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
+ * dataSize:int
+ * |||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||||
+ * | 
+ * | 
+ * | ...
+ * | 
+ * |
+ *
+ * the CellIndexWriter stores an array of longs using the BlockCompressedPayloadWriter
+ *
+ * logically this an array of longs
+ *
+ * |    0: start_0 : long
+ * |    1: start_1 : long
+ * |    ...
+ * |    n: start_n : long
+ * |    n+1: start_n + length_n : long  //ie, next position that would have been written to
+ *                                      //used again for invariant of length_i = row_i+1 - row_i
+ *
+ *      but this will be stored as block compressed. Reads are done by addressing it as a long array of bytes
+ *
+ * |    
+ * |    
+ * |
+ * |    
+ * |    
+ *
+ * resulting in
+ *
+ * |    
+ * | ----cell index------------------------
+ * |    
+ * |    
+ * |    
+ * |    
+ * | -------------------------------------
+ * |    
+ * | ----data stream------------------------
+ * |    
+ * |    
+ * |    
+ * |    
+ * | -------------------------------------
+ * 
+ */ + +public class CellWriter implements Serializer, Closeable +{ + private final IntSerializer intSerializer = new IntSerializer(); + private final CellIndexWriter cellIndexWriter; + private final BlockCompressedPayloadWriter payloadWriter; + + private CellWriter(CellIndexWriter cellIndexWriter, BlockCompressedPayloadWriter payloadWriter) + { + this.cellIndexWriter = cellIndexWriter; + this.payloadWriter = payloadWriter; + } + + public void write(byte[] cellBytes) throws IOException + { + if (cellBytes == null) { + cellIndexWriter.persistAndIncrement(0); + } else { + cellIndexWriter.persistAndIncrement(cellBytes.length); + payloadWriter.write(cellBytes); + } + } + + public void write(ByteBuffer cellByteBuffer) throws IOException + { + + if (cellByteBuffer == null) { + cellIndexWriter.persistAndIncrement(0); + } else { + cellIndexWriter.persistAndIncrement(cellByteBuffer.remaining()); + payloadWriter.write(cellByteBuffer); + } + } + + @Override + public void writeTo(WritableByteChannel channel, FileSmoosher smoosher) throws IOException + { + channel.write(intSerializer.serialize(cellIndexWriter.getSerializedSize())); + cellIndexWriter.writeTo(channel, smoosher); + channel.write(intSerializer.serialize(payloadWriter.getSerializedSize())); + payloadWriter.writeTo(channel, smoosher); + } + + @Override + public void close() throws IOException + { + cellIndexWriter.close(); + payloadWriter.close(); + } + + @Override + public long getSerializedSize() + { + return intSerializer.getSerializedSize() + + cellIndexWriter.getSerializedSize() + + intSerializer.getSerializedSize() + + payloadWriter.getSerializedSize(); + } + + public static class Builder + { + private final BlockCompressedPayloadWriter.Builder blockCompressedPayloadWriterBuilder; + + public Builder(NativeClearedByteBufferProvider byteBufferProvider, SegmentWriteOutMedium segmentWriteOutMedium) + { + blockCompressedPayloadWriterBuilder = + new BlockCompressedPayloadWriter.Builder(byteBufferProvider, segmentWriteOutMedium); + } + + public Builder setCompressionStrategy(CompressionStrategy compressionStrategy) + { + blockCompressedPayloadWriterBuilder.setCompressionStrategy(compressionStrategy); + + return this; + } + + public CellWriter build() throws IOException + { + BlockCompressedPayloadWriter cellIndexPayloadWriter = blockCompressedPayloadWriterBuilder.build(); + BlockCompressedPayloadWriter payloadWriter = blockCompressedPayloadWriterBuilder.build(); + CellIndexWriter cellIndexWriter = new CellIndexWriter(cellIndexPayloadWriter); + + return new CellWriter(cellIndexWriter, payloadWriter); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/DeserializingIOIterator.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/DeserializingIOIterator.java new file mode 100644 index 000000000000..656544657cd4 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/DeserializingIOIterator.java @@ -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.segment.serde.cell; + +import com.google.common.base.Preconditions; + +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.NoSuchElementException; + +public class DeserializingIOIterator implements IOIterator +{ + private static final int NEEDS_READ = -2; + private static final int EOF = -1; + + private final byte[] intBytes; + private final BufferedInputStream inputStream; + private final StagedSerde serde; + + private int nextSize; + + public DeserializingIOIterator(InputStream inputStream, StagedSerde serde) + { + this.inputStream = new BufferedInputStream(inputStream); + this.serde = serde; + intBytes = new byte[Integer.BYTES]; + nextSize = NEEDS_READ; + } + + @Override + public boolean hasNext() throws IOException + { + return getNextSize() > EOF; + } + + @Override + public T next() throws IOException + { + int currentNextSize = getNextSize(); + + if (currentNextSize == -1) { + throw new NoSuchElementException("end of buffer reached"); + } + + byte[] nextBytes = new byte[currentNextSize]; + int bytesRead = 0; + + while (bytesRead < currentNextSize) { + int result = inputStream.read(nextBytes, bytesRead, currentNextSize - bytesRead); + + if (result == -1) { + throw new NoSuchElementException("unexpected end of buffer reached"); + } + + bytesRead += result; + } + + Preconditions.checkState(bytesRead == currentNextSize); + T value = serde.deserialize(nextBytes); + + nextSize = NEEDS_READ; + + return value; + } + + private int getNextSize() throws IOException + { + if (nextSize == NEEDS_READ) { + int bytesRead = 0; + + while (bytesRead < Integer.BYTES) { + int result = inputStream.read(intBytes, bytesRead, Integer.BYTES - bytesRead); + + if (result == -1) { + nextSize = EOF; + return EOF; + } else { + bytesRead += result; + } + } + Preconditions.checkState(bytesRead == Integer.BYTES); + + nextSize = ByteBuffer.wrap(intBytes).order(ByteOrder.nativeOrder()).getInt(); + } + + return nextSize; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java new file mode 100644 index 000000000000..08669bdc3a14 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java @@ -0,0 +1,29 @@ +/* + * 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.segment.serde.cell; + +import java.io.IOException; + +public interface IOIterator +{ + boolean hasNext() throws IOException; + + T next() throws IOException; +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/IndexWriter.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/IndexWriter.java new file mode 100644 index 000000000000..46f26ad98861 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/IndexWriter.java @@ -0,0 +1,80 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; +import org.apache.druid.segment.writeout.WriteOutBytes; + +import java.io.IOException; +import java.nio.channels.WritableByteChannel; + +public class IndexWriter +{ + private final WriteOutBytes outBytes; + private final NumberSerializer positionSerializer; + private final NumberSerializer indexSizeSerializer; + + private boolean open = true; + private long position = 0; + + public IndexWriter( + WriteOutBytes outBytes, + NumberSerializer positionSerializer, + NumberSerializer indexSizeSerializer + ) + { + this.outBytes = outBytes; + this.positionSerializer = positionSerializer; + this.indexSizeSerializer = indexSizeSerializer; + } + + public IndexWriter(WriteOutBytes outBytes, NumberSerializer positionSerializer) + { + this(outBytes, positionSerializer, new IntSerializer()); + } + + public void persistAndIncrement(int increment) throws IOException + { + Preconditions.checkArgument(increment >= 0, "increment must be non-negative"); + Preconditions.checkState(open, "peristAndIncrement() must be called when open"); + outBytes.write(positionSerializer.serialize(position)); + position += increment; + } + + public void close() throws IOException + { + // when done, write an n+1'th entry for the next unused block; this lets us the use invariant + // of length of block i = entry i+1 - entry i for all i < n + outBytes.write(positionSerializer.serialize(position)); + open = false; + } + + public void transferTo(WritableByteChannel channel) throws IOException + { + channel.write(indexSizeSerializer.serialize(outBytes.size())); + outBytes.writeTo(channel); + } + + public long getSerializedSize() + { + return indexSizeSerializer.getSerializedSize() + Ints.checkedCast(outBytes.size()); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/IntIndexView.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/IntIndexView.java new file mode 100644 index 000000000000..fd40cd22d6a2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/IntIndexView.java @@ -0,0 +1,69 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.base.Preconditions; + +import java.nio.ByteBuffer; + +public class IntIndexView +{ + private final ByteBuffer byteBuffer; + private final int numberOfEntries; + + public IntIndexView(ByteBuffer byteBuffer) + { + this.byteBuffer = byteBuffer; + numberOfEntries = byteBuffer.remaining() / Integer.BYTES; + } + + public EntrySpan getEntrySpan(int entryNumber) + { + Preconditions.checkArgument( + entryNumber < numberOfEntries, "invalid entry number %s [%s]", entryNumber, numberOfEntries + ); + int start = byteBuffer.getInt(byteBuffer.position() + entryNumber * Integer.BYTES); + int nextStart = byteBuffer.getInt(byteBuffer.position() + ((entryNumber + 1) * Integer.BYTES)); + + return new EntrySpan(start, nextStart - start); + } + + public static class EntrySpan + { + private final int start; + private final int size; + + public EntrySpan(int start, int size) + { + this.start = start; + this.size = size; + } + + public int getStart() + { + return start; + } + + public int getSize() + { + return size; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/IntSerializer.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/IntSerializer.java new file mode 100644 index 000000000000..20e893500c62 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/IntSerializer.java @@ -0,0 +1,45 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.primitives.Ints; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class IntSerializer implements NumberSerializer +{ + private final ByteBuffer intValueByteBuffer = ByteBuffer.allocate(Integer.BYTES).order(ByteOrder.nativeOrder()); + + @Override + public ByteBuffer serialize(long value) + { + intValueByteBuffer.clear(); + intValueByteBuffer.putInt(Ints.checkedCast(value)).flip(); + + return intValueByteBuffer; + } + + @Override + public int getSerializedSize() + { + return Integer.BYTES; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/LongSerializer.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/LongSerializer.java new file mode 100644 index 000000000000..cfcaad5b8da9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/LongSerializer.java @@ -0,0 +1,43 @@ +/* + * 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.segment.serde.cell; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public class LongSerializer implements NumberSerializer +{ + private final ByteBuffer longValueByteBuffer = ByteBuffer.allocate(Long.BYTES).order(ByteOrder.nativeOrder()); + + @Override + public ByteBuffer serialize(long value) + { + longValueByteBuffer.clear(); + longValueByteBuffer.putLong(value).flip(); + + return longValueByteBuffer; + } + + @Override + public int getSerializedSize() + { + return Long.BYTES; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/NativeClearedByteBufferProvider.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/NativeClearedByteBufferProvider.java new file mode 100644 index 000000000000..5d861ef279c3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/NativeClearedByteBufferProvider.java @@ -0,0 +1,45 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.segment.CompressedPools; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.function.Supplier; + +/** + * supplies clear()'d ByteBuffers wrapped in a ResourceHolder + */ +public class NativeClearedByteBufferProvider implements Supplier> +{ + public static final NativeClearedByteBufferProvider INSTANCE = new NativeClearedByteBufferProvider(); + + @Override + public ResourceHolder get() + { + ResourceHolder holder = CompressedPools.getByteBuf(ByteOrder.nativeOrder()); + + holder.get().clear(); + + return holder; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/NumberSerializer.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/NumberSerializer.java new file mode 100644 index 000000000000..b4deafe548a3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/NumberSerializer.java @@ -0,0 +1,29 @@ +/* + * 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.segment.serde.cell; + +import java.nio.ByteBuffer; + +public interface NumberSerializer +{ + ByteBuffer serialize(long value); + + int getSerializedSize(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/PayloadEntrySpan.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/PayloadEntrySpan.java new file mode 100644 index 000000000000..86ff673af537 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/PayloadEntrySpan.java @@ -0,0 +1,42 @@ +/* + * 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.segment.serde.cell; + +public class PayloadEntrySpan +{ + private final long start; + private final int size; + + public PayloadEntrySpan(long start, int size) + { + this.start = start; + this.size = size; + } + + public long getStart() + { + return start; + } + + public int getSize() + { + return size; + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/StagedSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/StagedSerde.java new file mode 100644 index 000000000000..15e7ad980eb2 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/StagedSerde.java @@ -0,0 +1,64 @@ +/* + * 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.segment.serde.cell; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; + +public interface StagedSerde +{ + /** + * Useful method when some computation is necessary to prepare for serialization without actually writing out + * all the bytes in order to determine the serialized size. It allows encapsulation of the size computation and + * the final logical to actually store into a ByteBuffer. It also allows for callers to pack multiple serialized + * objects into a single ByteBuffer without extra copies of a byte[]/ByteBuffer by using the {@link StorableBuffer} + * instance returned + * + * @param value - object to serialize + * @return an object that reports its serialized size and how to serialize the object to a ByteBuffer + */ + StorableBuffer serializeDelayed(@Nullable T value); + + /** + * Default implementation for when a byte[] is desired. Typically, this default should suffice. Implementing + * serializeDelayed() includes the logic of how to store into a ByteBuffer + * + * @param value - object to serialize + * @return serialized byte[] of value + */ + default byte[] serialize(T value) + { + StorableBuffer storableBuffer = serializeDelayed(value); + ByteBuffer byteBuffer = ByteBuffer.allocate(storableBuffer.getSerializedSize()).order(ByteOrder.nativeOrder()); + + storableBuffer.store(byteBuffer); + + return byteBuffer.array(); + } + + @Nullable + T deserialize(ByteBuffer byteBuffer); + + default T deserialize(byte[] bytes) + { + return deserialize(ByteBuffer.wrap(bytes).order(ByteOrder.nativeOrder())); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/StorableBuffer.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/StorableBuffer.java new file mode 100644 index 000000000000..870d531c297e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/StorableBuffer.java @@ -0,0 +1,48 @@ +/* + * 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.segment.serde.cell; + +import java.nio.ByteBuffer; + +/** + * useful when work needs to be done to prepare for serialization (eg encoding) which is also necessary + * to know how large a buffer is needed. Hence, returns both the size and a method to store in a buffer + * caller must allocate of sufficient size + */ +public interface StorableBuffer +{ + StorableBuffer EMPTY = new StorableBuffer() + { + @Override + public void store(ByteBuffer byteBuffer) + { + } + + @Override + public int getSerializedSize() + { + return 0; + } + }; + + void store(ByteBuffer byteBuffer); + + int getSerializedSize(); +} diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStoreTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStoreTest.java new file mode 100644 index 000000000000..ec9ff157759e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringBufferStoreTest.java @@ -0,0 +1,382 @@ +/* + * 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.primitives.Ints; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.serde.cell.IOIterator; +import org.apache.druid.segment.serde.cell.NativeClearedByteBufferProvider; +import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes; +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Random; + +public class SerializablePairLongStringBufferStoreTest +{ + private final Random random = new Random(0); + private static final int MIN_INTEGER = 100; + private static final long MIN_LONG = 0L; + private final SerializablePairLongString[] integerRangeArr = new SerializablePairLongString[]{ + new SerializablePairLongString((long) MIN_INTEGER, "fuu"), + new SerializablePairLongString(101L, "bar"), + new SerializablePairLongString(102L, "baz"), + }; + private final SerializablePairLongString[] longRangeArr = new SerializablePairLongString[]{ + new SerializablePairLongString(MIN_LONG, "fuu"), + new SerializablePairLongString(100L, "bar"), + new SerializablePairLongString((long) Integer.MAX_VALUE, "baz"), + new SerializablePairLongString(Long.MAX_VALUE, "fuubarbaz"), + }; + + private final SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium(); + + private SerializablePairLongStringBufferStore bufferStore; + + @Before + public void setup() throws Exception + { + bufferStore = new SerializablePairLongStringBufferStore(writeOutMedium.makeWriteOutBytes()); + } + + @Test + public void testIteratorSimple() throws Exception + { + for (SerializablePairLongString value : integerRangeArr) { + bufferStore.store(value); + } + + IOIterator iterator = bufferStore.iterator(); + + int i = 0; + while (iterator.hasNext()) { + Assert.assertEquals(integerRangeArr[i], iterator.next()); + i++; + } + } + + @Test + public void testIteratorEmptyBuffer() throws Exception + { + IOIterator iterator = bufferStore.iterator(); + + Assert.assertFalse(iterator.hasNext()); + } + + @Test + public void testIteratorNull() throws Exception + { + bufferStore.store(null); + IOIterator iterator = bufferStore.iterator(); + Assert.assertTrue(iterator.hasNext()); + Assert.assertNull(iterator.next()); + } + + @Test + public void testIteratorIdempotentHasNext() throws Exception + { + bufferStore.store(integerRangeArr[0]); + + IOIterator iterator = bufferStore.iterator(); + + Assert.assertTrue(iterator.hasNext()); + // expect hasNext() to not modify state + Assert.assertTrue(iterator.hasNext()); + } + + @Test(expected = NoSuchElementException.class) + public void testIteratorEmptyThrows() throws Exception + { + IOIterator iterator = bufferStore.iterator(); + iterator.next(); + } + + @Test + public void testIteratorEmptyHasNext() throws Exception + { + IOIterator iterator = bufferStore.iterator(); + Assert.assertFalse(iterator.hasNext()); + } + + @Test + public void testMinValueUsesInteger() throws Exception + { + for (SerializablePairLongString value : integerRangeArr) { + bufferStore.store(value); + } + + SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader(); + Assert.assertEquals(integerRangeArr[0].lhs.longValue(), columnHeader.getMinValue()); + Assert.assertTrue(columnHeader.isUseIntegerDeltas()); + } + + @Test + public void testMinValueUsesLong() throws Exception + { + for (SerializablePairLongString value : longRangeArr) { + bufferStore.store(value); + } + + SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader(); + Assert.assertEquals(MIN_LONG, columnHeader.getMinValue()); + Assert.assertFalse(columnHeader.isUseIntegerDeltas()); + } + + @Test + public void testMinValueUsesIntegerSerialization() throws Exception + { + for (SerializablePairLongString value : integerRangeArr) { + bufferStore.store(value); + } + + SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader(); + + HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes(); + try (ResourceHolder resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) { + columnHeader.transferTo(channel); + + ByteBuffer byteBuffer = resourceHolder.get(); + channel.writeTo(byteBuffer); + byteBuffer.flip(); + + SerializablePairLongStringColumnHeader deserializedColumnhHeader = + SerializablePairLongStringColumnHeader.fromBuffer(byteBuffer); + Assert.assertEquals(MIN_INTEGER, deserializedColumnhHeader.getMinValue()); + Assert.assertTrue(deserializedColumnhHeader.isUseIntegerDeltas()); + } + } + + @Test + public void testMinValueSerialization() throws Exception + + { + for (SerializablePairLongString value : longRangeArr) { + bufferStore.store(value); + } + + SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader(); + + HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes(); + try (ResourceHolder resourceHolder = NativeClearedByteBufferProvider.INSTANCE.get()) { + columnHeader.transferTo(channel); + + ByteBuffer byteBuffer = resourceHolder.get(); + + channel.writeTo(byteBuffer); + byteBuffer.flip(); + + SerializablePairLongStringColumnHeader deserializedColumnhHeader = + SerializablePairLongStringColumnHeader.fromBuffer(byteBuffer); + Assert.assertEquals(MIN_LONG, deserializedColumnhHeader.getMinValue()); + Assert.assertFalse(deserializedColumnhHeader.isUseIntegerDeltas()); + } + } + + @Test + public void testVariedSize() throws Exception + { + int rowCount = 100; + int maxStringSize = 1024 * 1024; + int minStringSize = 1024; + List input = new ArrayList<>(rowCount); + int totalCount = 0; + + for (int i = 0; i < rowCount; i++) { + long longValue = random.nextLong(); + SerializablePairLongString value = + new SerializablePairLongString(longValue, RandomStringUtils.randomAlphabetic(minStringSize, maxStringSize)); + + input.add(value); + totalCount += longValue; + totalCount = Math.max(totalCount, 0); + + bufferStore.store(value); + } + + IOIterator iterator = bufferStore.iterator(); + int i = 0; + + while (iterator.hasNext()) { + Assert.assertEquals(input.get(i), iterator.next()); + i++; + } + } + + @Test + public void testLargeBuffer() throws Exception + { + // note: tests single element larger than 64k + int stringSize = 128 * 1024; + SerializablePairLongString value = + new SerializablePairLongString(Long.MAX_VALUE, RandomStringUtils.randomAlphabetic(stringSize)); + + bufferStore.store(value); + + IOIterator iterator = bufferStore.iterator(); + + Assert.assertTrue(iterator.hasNext()); + Assert.assertEquals(value, iterator.next()); + Assert.assertFalse(iterator.hasNext()); + } + + @Test + public void testLargeValueCount() throws Exception + { + List valueList = new ArrayList<>(); + + for (int i = 0; i < 10000; i++) { + valueList.add(new SerializablePairLongString(Integer.MAX_VALUE + (long) i, "the same string")); + } + + assertBufferedValuesEqual(valueList); + } + + @Test + public void testOverflowTransfer() throws Exception + { + bufferStore.store(new SerializablePairLongString(Long.MIN_VALUE, "fuu")); + bufferStore.store(new SerializablePairLongString(Long.MAX_VALUE, "fuu")); + + SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader(); + + Assert.assertEquals(0, columnHeader.getMinValue()); + + SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter( + NativeClearedByteBufferProvider.INSTANCE, + writeOutMedium + ); + + Assert.assertEquals(94, transferredBuffer.getSerializedSize()); + } + + @Test + public void testNullOnlyTransfer() throws Exception + { + bufferStore.store(null); + bufferStore.store(null); + bufferStore.store(null); + + SerializablePairLongStringColumnHeader columnHeader = bufferStore.createColumnHeader(); + + Assert.assertEquals(0, columnHeader.getMinValue()); + + SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer = bufferStore.transferToRowWriter( + NativeClearedByteBufferProvider.INSTANCE, + writeOutMedium + ); + + Assert.assertEquals(59, transferredBuffer.getSerializedSize()); + } + + @Test + public void testTransferIntegerRange() throws Exception + { + for (SerializablePairLongString value : integerRangeArr) { + bufferStore.store(value); + } + + Assert.assertTrue(bufferStore.createColumnHeader().isUseIntegerDeltas()); + + assertTransferredValuesEqual(integerRangeArr); + } + + @Test + public void testTransferLongRange() throws Exception + { + for (SerializablePairLongString value : longRangeArr) { + bufferStore.store(value); + } + + Assert.assertFalse(bufferStore.createColumnHeader().isUseIntegerDeltas()); + + assertTransferredValuesEqual(longRangeArr); + } + + private void assertBufferedValuesEqual(List input) throws IOException + { + for (SerializablePairLongString pairLongString : input) { + bufferStore.store(pairLongString); + } + + IOIterator iterator = bufferStore.iterator(); + int i = 0; + + while (iterator.hasNext()) { + Assert.assertEquals(input.get(i), iterator.next()); + i++; + } + + Assert.assertEquals( + StringUtils.format("element count mismatch: expected %s, got %s", input.size(), i), + input.size(), + i + ); + } + + private void assertTransferredValuesEqual(SerializablePairLongString[] input) throws IOException + { + SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer = + bufferStore.transferToRowWriter(NativeClearedByteBufferProvider.INSTANCE, writeOutMedium); + HeapByteBufferWriteOutBytes resultChannel = new HeapByteBufferWriteOutBytes(); + + transferredBuffer.writeTo(resultChannel, null); + + try (SerializablePairLongStringComplexColumn column = createComplexColumn(transferredBuffer, resultChannel)) { + for (int i = 0; i < input.length; i++) { + Assert.assertEquals(input[i], column.getRowValue(i)); + } + } + } + + private static SerializablePairLongStringComplexColumn createComplexColumn( + SerializablePairLongStringBufferStore.TransferredBuffer transferredBuffer, + HeapByteBufferWriteOutBytes resultChannel + ) + { + ByteBuffer byteBuffer = ByteBuffer.allocate(Ints.checkedCast(transferredBuffer.getSerializedSize())); + + resultChannel.readFully(0, byteBuffer); + byteBuffer.flip(); + + SerializablePairLongStringComplexMetricSerde complexMetricSerde = new SerializablePairLongStringComplexMetricSerde(); + ColumnBuilder builder = new ColumnBuilder(); + + complexMetricSerde.deserializeColumn(byteBuffer, builder); + builder.setType(ValueType.COMPLEX); + + ColumnHolder columnHolder = builder.build(); + SerializablePairLongStringComplexColumn column = (SerializablePairLongStringComplexColumn) columnHolder.getColumn(); + + return column; + } +} diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java new file mode 100644 index 000000000000..c8605f4695b8 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringComplexMetricSerdeTest.java @@ -0,0 +1,237 @@ +/* + * 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.collect.ImmutableList; +import org.apache.druid.segment.GenericColumnSerializer; +import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.serde.cell.RandomStringUtils; +import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes; +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; +import org.apache.druid.segment.writeout.SegmentWriteOutMedium; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.stream.Collectors; + +public class SerializablePairLongStringComplexMetricSerdeTest +{ + private static final SerializablePairLongStringComplexMetricSerde COMPLEX_METRIC_SERDE = + new SerializablePairLongStringComplexMetricSerde(); + + // want deterministic test input + private final Random random = new Random(0); + private final RandomStringUtils randomStringUtils = new RandomStringUtils(random); + + private GenericColumnSerializer serializer; + + @SuppressWarnings("unchecked") + @Before + public void setup() + { + SegmentWriteOutMedium writeOutMedium = new OnHeapMemorySegmentWriteOutMedium(); + serializer = (GenericColumnSerializer) COMPLEX_METRIC_SERDE.getSerializer( + writeOutMedium, + "not-used" + ); + } + + @Test + public void testSingle() throws Exception + { + assertExpected(ImmutableList.of(new SerializablePairLongString(100L, "fuu")), 77); + } + + @Test + public void testLargeString() throws Exception + { + // single entry spans more than one block in underlying storage + assertExpected(ImmutableList.of(new SerializablePairLongString( + 100L, + randomStringUtils.randomAlphanumeric(2 * 1024 * 1024) + )), 2103140); + } + + @Test + public void testCompressable() throws Exception + { + int numStrings = 10; + List valueList = new ArrayList<>(); + List stringList = new ArrayList<>(); + + for (int i = 0; i < numStrings; i++) { + stringList.add(randomStringUtils.randomAlphanumeric(1024)); + } + for (int i = 0; i < 10000; i++) { + valueList.add(new SerializablePairLongString(Integer.MAX_VALUE + (long) i, stringList.get(i % numStrings))); + } + + //actual input bytes in naive encoding is ~10mb + assertExpected(valueList, 1746026); + } + + @Test + public void testHighlyCompressable() throws Exception + { + List valueList = new ArrayList<>(); + + String stringValue = randomStringUtils.randomAlphanumeric(1024); + for (int i = 0; i < 10000; i++) { + valueList.add(new SerializablePairLongString(Integer.MAX_VALUE + (long) i, stringValue)); + } + + //actual input bytes in naive encoding is ~10mb + assertExpected(valueList, 289645); + } + + @Test + public void testRandom() throws Exception + { + List valueList = new ArrayList<>(); + + for (int i = 0; i < 10000; i++) { + valueList.add(new SerializablePairLongString(random.nextLong(), randomStringUtils.randomAlphanumeric(1024))); + } + + assertExpected(valueList, 10428975); + } + + @Test + public void testNullString() throws Exception + { + assertExpected(ImmutableList.of(new SerializablePairLongString(100L, null)), 74); + } + + @Test + public void testEmpty() throws Exception + { + // minimum size for empty data + assertExpected(Collections.emptyList(), 57); + } + + @Test + public void testSingleNull() throws Exception + { + assertExpected(Arrays.asList(new SerializablePairLongString[]{null}), 58); + } + + @Test + public void testMultipleNull() throws Exception + { + assertExpected(Arrays.asList(null, null, null, null), 59); + } + + private void assertExpected(List expected) throws IOException + { + assertExpected(expected, -1); + } + + private void assertExpected(List expected, int expectedSize) throws IOException + { + List valueSelectors = + expected.stream().map(SerializablePairLongStringValueSelector::new).collect(Collectors.toList()); + ByteBuffer byteBuffer = serializeAllValuesToByteBuffer(valueSelectors, serializer, expectedSize); + + try (SerializablePairLongStringComplexColumn complexColumn = createComplexColumn(byteBuffer)) { + for (int i = 0; i < valueSelectors.size(); i++) { + Assert.assertEquals(expected.get(i), complexColumn.getRowValue(i)); + } + } + } + + private SerializablePairLongStringComplexColumn createComplexColumn(ByteBuffer byteBuffer) + { + ColumnBuilder builder = new ColumnBuilder(); + int serializedSize = byteBuffer.remaining(); + + COMPLEX_METRIC_SERDE.deserializeColumn(byteBuffer, builder); + builder.setType(ValueType.COMPLEX); + + ColumnHolder columnHolder = builder.build(); + + SerializablePairLongStringComplexColumn column = (SerializablePairLongStringComplexColumn) columnHolder.getColumn(); + + Assert.assertEquals(serializedSize, column.getLength()); + Assert.assertEquals("serializablePairLongString", column.getTypeName()); + Assert.assertEquals(SerializablePairLongString.class, column.getClazz()); + + return column; + } + + + private static ByteBuffer serializeAllValuesToByteBuffer( + Collection valueSelectors, + GenericColumnSerializer serializer, + int expectedSize + ) throws IOException + { + serializer.open(); + + for (SerializablePairLongStringValueSelector valueSelector : valueSelectors) { + serializer.serialize(valueSelector); + } + + return serializeToByteBuffer(serializer, expectedSize); + } + + private static ByteBuffer serializeToByteBuffer( + GenericColumnSerializer serializer, + int expectedSize + ) throws IOException + { + HeapByteBufferWriteOutBytes channel = new HeapByteBufferWriteOutBytes(); + + serializer.writeTo(channel, null); + + ByteBuffer byteBuffer = ByteBuffer.allocate((int) channel.size()).order(ByteOrder.nativeOrder()); + + channel.readFully(0, byteBuffer); + byteBuffer.flip(); + + if (expectedSize > -1) { + Assert.assertEquals(expectedSize, serializer.getSerializedSize()); + } + + Assert.assertEquals(serializer.getSerializedSize(), byteBuffer.limit()); + + return byteBuffer; + } + + private static class SerializablePairLongStringValueSelector + extends SingleValueColumnValueSelector + { + public SerializablePairLongStringValueSelector(SerializablePairLongString value) + { + super(SerializablePairLongString.class, value); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerdeTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerdeTest.java new file mode 100644 index 000000000000..d0489cf92814 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringDeltaEncodedStagedSerdeTest.java @@ -0,0 +1,99 @@ +/* + * 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.segment.serde.cell.RandomStringUtils; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; + +public class SerializablePairLongStringDeltaEncodedStagedSerdeTest +{ + private static final SerializablePairLongStringDeltaEncodedStagedSerde INTEGER_SERDE = + new SerializablePairLongStringDeltaEncodedStagedSerde(0L, true); + + private static final SerializablePairLongStringDeltaEncodedStagedSerde LONG_SERDE = + new SerializablePairLongStringDeltaEncodedStagedSerde(0L, false); + + private final RandomStringUtils randomStringUtils = new RandomStringUtils(); + + @Test + public void testNull() + { + assertValueEquals(null, 0, INTEGER_SERDE); + } + + @Test + public void testSimpleInteger() + { + assertValueEquals(new SerializablePairLongString(100L, "fuu"), 11, INTEGER_SERDE); + } + + @Test + public void testNullStringInteger() + { + assertValueEquals(new SerializablePairLongString(100L, null), 8, INTEGER_SERDE); + } + + @Test + public void testLargeStringInteger() + { + assertValueEquals( + new SerializablePairLongString(100L, randomStringUtils.randomAlphanumeric(1024 * 1024)), + 1048584, + INTEGER_SERDE + ); + } + + @Test + public void testSimpleLong() + { + assertValueEquals(new SerializablePairLongString(100L, "fuu"), 15, LONG_SERDE); + } + + @Test + public void testNullStringLong() + { + assertValueEquals(new SerializablePairLongString(100L, null), 12, LONG_SERDE); + } + + @Test + public void testLargeStringLong() + { + assertValueEquals( + new SerializablePairLongString(100L, randomStringUtils.randomAlphanumeric(10 * 1024 * 1024)), + 10485772, + LONG_SERDE + ); + } + + private static void assertValueEquals( + @Nullable SerializablePairLongString value, + int size, + SerializablePairLongStringDeltaEncodedStagedSerde serde + ) + { + byte[] bytes = serde.serialize(value); + Assert.assertEquals(size, bytes.length); + SerializablePairLongString deserialized = serde.deserialize(bytes); + Assert.assertEquals(value, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerdeTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerdeTest.java new file mode 100644 index 000000000000..23d57f0aa9ed --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SerializablePairLongStringSimpleStagedSerdeTest.java @@ -0,0 +1,70 @@ +/* + * 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.segment.serde.cell.RandomStringUtils; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nullable; +import java.util.Random; + +public class SerializablePairLongStringSimpleStagedSerdeTest +{ + private static final SerializablePairLongStringSimpleStagedSerde SERDE = + new SerializablePairLongStringSimpleStagedSerde(); + + private final RandomStringUtils randomStringUtils = new RandomStringUtils(new Random(0)); + + @Test + public void testSimple() + { + assertValueEquals(new SerializablePairLongString(Long.MAX_VALUE, "fuu"), 15); + } + + @Test + public void testNull() + { + assertValueEquals(null, 0); + } + + @Test + public void testNullString() + { + assertValueEquals(new SerializablePairLongString(Long.MAX_VALUE, null), 12); + } + + @Test + public void testLargeString() + { + assertValueEquals( + new SerializablePairLongString(Long.MAX_VALUE, randomStringUtils.randomAlphanumeric(1024 * 1024)), + 1048588 + ); + } + + private static void assertValueEquals(@Nullable SerializablePairLongString value, int size) + { + byte[] bytes = SERDE.serialize(value); + Assert.assertEquals(size, bytes.length); + SerializablePairLongString deserialized = SERDE.deserialize(bytes); + Assert.assertEquals(value, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/SingleValueColumnValueSelector.java b/processing/src/test/java/org/apache/druid/query/aggregation/SingleValueColumnValueSelector.java new file mode 100644 index 000000000000..8c9d232ca9d7 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/aggregation/SingleValueColumnValueSelector.java @@ -0,0 +1,79 @@ +/* + * 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.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +public class SingleValueColumnValueSelector implements ColumnValueSelector +{ + private final Class valueClass; + private final T value; + + public SingleValueColumnValueSelector(Class valueClass, T value) + { + this.valueClass = valueClass; + this.value = value; + } + + @Override + public double getDouble() + { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat() + { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong() + { + throw new UnsupportedOperationException(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + } + + @Override + public boolean isNull() + { + return false; + } + + @Nullable + @Override + public T getObject() + { + return value; + } + + @Override + public Class classOfObject() + { + return valueClass; + } +} diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java index 250525ca1b78..267fa52ab639 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/first/StringFirstTimeseriesQueryTest.java @@ -30,7 +30,7 @@ import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.SerializablePairLongString; -import org.apache.druid.query.aggregation.SerializablePairLongStringSerde; +import org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerde; import org.apache.druid.query.timeseries.DefaultTimeseriesQueryMetrics; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; @@ -68,7 +68,7 @@ public class StringFirstTimeseriesQueryTest extends InitializedNullHandlingTest @Before public void setUp() throws IndexSizeExceededException { - final SerializablePairLongStringSerde serde = new SerializablePairLongStringSerde(); + final SerializablePairLongStringComplexMetricSerde serde = new SerializablePairLongStringComplexMetricSerde(); ComplexMetrics.registerSerde(serde.getTypeName(), serde); incrementalIndex = new OnheapIncrementalIndex.Builder() diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java index 6c017bab4af0..33bff8146ff6 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/last/StringLastTimeseriesQueryTest.java @@ -30,7 +30,7 @@ import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.SerializablePairLongString; -import org.apache.druid.query.aggregation.SerializablePairLongStringSerde; +import org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerde; import org.apache.druid.query.timeseries.DefaultTimeseriesQueryMetrics; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; @@ -67,7 +67,7 @@ public class StringLastTimeseriesQueryTest @Before public void setUp() throws IndexSizeExceededException { - final SerializablePairLongStringSerde serde = new SerializablePairLongStringSerde(); + final SerializablePairLongStringComplexMetricSerde serde = new SerializablePairLongStringComplexMetricSerde(); ComplexMetrics.registerSerde(serde.getTypeName(), serde); incrementalIndex = new OnheapIncrementalIndex.Builder() diff --git a/processing/src/test/java/org/apache/druid/segment/serde/ComplexMetricsTest.java b/processing/src/test/java/org/apache/druid/segment/serde/ComplexMetricsTest.java index df580a5c3abb..d9e6b7ba54c0 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/ComplexMetricsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/ComplexMetricsTest.java @@ -19,7 +19,7 @@ package org.apache.druid.segment.serde; -import org.apache.druid.query.aggregation.SerializablePairLongStringSerde; +import org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerde; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import org.junit.Assert; import org.junit.Rule; @@ -67,9 +67,9 @@ public void testConflicting() Assert.assertTrue(serde instanceof HyperUniquesSerde); expectedException.expect(IllegalStateException.class); - expectedException.expectMessage("Incompatible serializer for type[hyperUnique] already exists. Expected [org.apache.druid.query.aggregation.SerializablePairLongStringSerde], found [org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde"); + expectedException.expectMessage("Incompatible serializer for type[hyperUnique] already exists. Expected [org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerde], found [org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde"); - ComplexMetrics.registerSerde("hyperUnique", new SerializablePairLongStringSerde()); + ComplexMetrics.registerSerde("hyperUnique", new SerializablePairLongStringComplexMetricSerde()); serde = ComplexMetrics.getSerdeForType("hyperUnique"); Assert.assertNotNull(serde); diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterTest.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterTest.java new file mode 100644 index 000000000000..aa1541059ee0 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterTest.java @@ -0,0 +1,56 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; + +public class BlockCompressedPayloadWriterTest extends BytesReadWriteTestBase +{ + public BlockCompressedPayloadWriterTest() + { + super( + new BlockCompressedPayloadWriterToBytesWriter.Builder( + new BlockCompressedPayloadWriter.Builder( + NativeClearedByteBufferProvider.INSTANCE, + new OnHeapMemorySegmentWriteOutMedium() + ) + ), + ByteWriterTestHelper.ValidationFunctionBuilder.PAYLOAD_WRITER_VALIDATION_FUNCTION_FACTORY, + new BytesReadWriteTestCases() + .setTestCaseValue(BytesReadWriteTest::testSingleWriteBytes, TestCaseResult.of(4115)) + .setTestCaseValue(BytesReadWriteTest::testSingleMultiBlockWriteBytes, TestCaseResult.of(1049169)) + .setTestCaseValue(BytesReadWriteTest::testSingleMultiBlockWriteBytesWithPrelude, TestCaseResult.of(1053238)) + // BytesReadWriteTest::testEmptyByteArray -> compression header is 12-bytes + .setTestCaseValue(BytesReadWriteTest::testEmptyByteArray, TestCaseResult.of(12)) + .setTestCaseValue(BytesReadWriteTest::testNull, TestCaseResult.of(new NullPointerException())) + .setTestCaseValue(BytesReadWriteTest::testSingleLong, TestCaseResult.of(25)) + .setTestCaseValue(BytesReadWriteTest::testVariableSizedCompressablePayloads, TestCaseResult.of(1180)) + .setTestCaseValue( + BytesReadWriteTest::testOutliersInNormalDataUncompressablePayloads, + TestCaseResult.of(574302) + ) + .setTestCaseValue(BytesReadWriteTest::testOutliersInNormalDataCompressablePayloads, TestCaseResult.of(5997)) + .setTestCaseValue(BytesReadWriteTest::testSingleUncompressableBlock, TestCaseResult.of(65715)) + .setTestCaseValue(BytesReadWriteTest::testSingleWriteByteBufferZSTD, TestCaseResult.of(796)) + .setTestCaseValue(BytesReadWriteTest::testRandomBlockAccess, TestCaseResult.of(3124842)) + + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterToBytesWriter.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterToBytesWriter.java new file mode 100644 index 000000000000..c8debdc57e68 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/BlockCompressedPayloadWriterToBytesWriter.java @@ -0,0 +1,90 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.segment.data.CompressionStrategy; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +public class BlockCompressedPayloadWriterToBytesWriter implements BytesWriter +{ + private final BlockCompressedPayloadWriter blockCompressedPayloadWriter; + + public BlockCompressedPayloadWriterToBytesWriter(BlockCompressedPayloadWriter blockCompressedPayloadWriter) + { + this.blockCompressedPayloadWriter = blockCompressedPayloadWriter; + } + + @Override + public void write(byte[] payload) throws IOException + { + blockCompressedPayloadWriter.write(payload); + } + + @Override + public void write(ByteBuffer payload) throws IOException + { + blockCompressedPayloadWriter.write(payload); + } + + @Override + public void close() throws IOException + { + blockCompressedPayloadWriter.close(); + } + + @Override + public void transferTo(WritableByteChannel channel) throws IOException + { + blockCompressedPayloadWriter.writeTo(channel, null); + } + + @Override + public long getSerializedSize() + { + return blockCompressedPayloadWriter.getSerializedSize(); + } + + public static class Builder implements BytesWriterBuilder + { + private final BlockCompressedPayloadWriter.Builder builder; + + public Builder(BlockCompressedPayloadWriter.Builder builder) + { + this.builder = builder; + } + + @Override + public BytesWriter build() throws IOException + { + return new BlockCompressedPayloadWriterToBytesWriter(builder.build()); + } + + @Override + public BytesWriterBuilder setCompressionStrategy(CompressionStrategy compressionStrategy) + { + builder.setCompressionStrategy(compressionStrategy); + + return this; + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/ByteWriterTestHelper.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/ByteWriterTestHelper.java new file mode 100644 index 000000000000..29b015368473 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/ByteWriterTestHelper.java @@ -0,0 +1,372 @@ +/* + * 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.segment.serde.cell; + +import com.google.common.primitives.Ints; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.writeout.HeapByteBufferWriteOutBytes; +import org.junit.Assert; + +import javax.annotation.Nonnull; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +public class ByteWriterTestHelper +{ + private static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new byte[0]); + + private final BytesWriterBuilder bytesWriterBuilder; + private final ByteBuffer uncompressedBlock1; + private final ByteBuffer uncompressedBlock2; + private final ValidationFunctionBuilder validationFunctionBuilder; + private CompressionStrategy compressionStrategy = CompressionStrategy.LZ4; + + public ByteWriterTestHelper( + BytesWriterBuilder bytesWriterBuilder, + ByteBuffer uncompressedBlock1, + ByteBuffer uncompressedBlock2, + ValidationFunctionBuilder validationFunctionBuilder + ) + { + this.bytesWriterBuilder = bytesWriterBuilder; + this.uncompressedBlock1 = uncompressedBlock1; + this.uncompressedBlock2 = uncompressedBlock2; + this.validationFunctionBuilder = validationFunctionBuilder; + } + + public ByteWriterTestHelper setCompressionStrategy(CompressionStrategy compressionStrategy) + { + this.compressionStrategy = compressionStrategy; + bytesWriterBuilder.setCompressionStrategy(compressionStrategy); + + return this; + } + + public ByteBuffer writePayloadAsByteArray(ByteBuffer payload) throws IOException + { + return writePayload(payload, BufferWriterAsBytes.INSTANCE); + } + + public ByteBuffer writePayloadAsByteBuffer(ByteBuffer payload) throws IOException + { + return writePayload(payload, BufferWriterAsBuffer.INSTANCE); + } + + public List generateRaggedPayloadBuffer( + int baseMin, + int baseMax, + int stepSize, + int largeSize, + int largeCount + ) + { + return generateRaggedPayloadBuffer(baseMin, baseMax, stepSize, largeSize, largeCount, Integer.MAX_VALUE); + } + + public List generateRaggedPayloadBuffer( + int baseMin, + int baseMax, + int stepSize, + int largeSize, + int largeCount, + int modulo + ) + { + List byteBufferList = new ArrayList<>(); + + for (int i = baseMin; i < baseMax; i += stepSize) { + byteBufferList.add(generateIntPayloads(baseMin + i, modulo)); + } + + for (int j = 0; j < largeCount; j++) { + byteBufferList.add(generateIntPayloads(largeSize, modulo)); + + for (int i = baseMin; i < baseMax; i += stepSize) { + byteBufferList.add(generateIntPayloads(baseMin + i, modulo)); + } + } + + return byteBufferList; + } + + public void validateRead(List byteBufferList) throws Exception + { + ValidationFunction validationFunction = validationFunctionBuilder.create(this); + validationFunction.validateBufferList(byteBufferList); + } + + public void validateReadAndSize(List byteBufferList, int expectedSize) throws Exception + { + ValidationFunction validationFunction = validationFunctionBuilder.create(this); + ByteBuffer masterByteBuffer = validationFunction.validateBufferList(byteBufferList); + int actualSize = masterByteBuffer.limit(); + + if (expectedSize > -1) { + Assert.assertEquals(expectedSize, actualSize); + } + } + + public ByteBuffer writePayload(ByteBuffer sourcePayLoad, BufferWriter bufferWriter) throws IOException + { + return writePayloadList(Collections.singletonList(sourcePayLoad), bufferWriter); + } + + public ByteBuffer writePayloadList(List payloadList) throws IOException + { + return writePayloadList(payloadList, BufferWriterAsBuffer.INSTANCE); + } + + public ByteBuffer writePayloadList(List payloadList, BufferWriter bufferWriter) throws IOException + { + BytesWriter bytesWriter = bytesWriterBuilder.build(); + + try { + for (ByteBuffer payload : payloadList) { + bufferWriter.writeTo(bytesWriter, payload); + } + } + finally { + bytesWriter.close(); + } + + + HeapByteBufferWriteOutBytes bufferWriteOutBytes = new HeapByteBufferWriteOutBytes(); + + bytesWriter.transferTo(bufferWriteOutBytes); + + int payloadSerializedSize = Ints.checkedCast(bytesWriter.getSerializedSize()); + ByteBuffer masterByteBuffer = ByteBuffer.allocate(payloadSerializedSize).order(ByteOrder.nativeOrder()); + + bufferWriteOutBytes.readFully(0, masterByteBuffer); + masterByteBuffer.flip(); + + Assert.assertEquals(bytesWriter.getSerializedSize(), masterByteBuffer.limit()); + + return masterByteBuffer; + } + + public ByteBuffer generateIntPayloads(int intCount) + { + return generateIntPayloads(intCount, Integer.MAX_VALUE); + } + + public ByteBuffer generateIntPayloads(int intCount, int modulo) + { + ByteBuffer payload = ByteBuffer.allocate(Integer.BYTES * intCount).order(ByteOrder.nativeOrder()); + + for (int i = intCount - 1; i >= 0; i--) { + + payload.putInt(i % modulo); + } + + payload.flip(); + + return payload; + } + + @Nonnull + public ByteBuffer generateBufferWithLongs(int longCount) + { + ByteBuffer longPayload = ByteBuffer.allocate(Long.BYTES * longCount).order(ByteOrder.nativeOrder()); + + for (int i = 0; i < longCount; i++) { + longPayload.putLong(longCount - i - 1); + } + + longPayload.flip(); + + return longPayload; + } + + public ByteBuffer validateBufferWriteAndReadBlockCompressed(List bufferList, boolean useRandom) + throws IOException + { + long position = 0; + List payloadReadList = new ArrayList<>(); + + for (ByteBuffer byteBuffer : bufferList) { + int expectedSize = byteBuffer == null ? 0 : byteBuffer.limit(); + payloadReadList.add(new PayloadEntrySpan(position, expectedSize)); + position += expectedSize; + } + + ByteBuffer masterByteBuffer = writePayloadList(bufferList, new BufferWriterAsBytes()); + BlockCompressedPayloadReader payloadReader = BlockCompressedPayloadReader.create( + masterByteBuffer, + uncompressedBlock1, + compressionStrategy.getDecompressor() + ); + List positions = new ArrayList<>(bufferList.size()); + + for (int i = 0; i < bufferList.size(); i++) { + positions.add(i); + } + + Random random = new Random(0); + + if (useRandom) { + Collections.shuffle(positions, random); + } + + for (int index : positions) { + ByteBuffer expectedByteBuffer = bufferList.get(index); + PayloadEntrySpan payloadEntrySpan = payloadReadList.get(index); + ByteBuffer readByteBuffer = payloadReader.read(payloadEntrySpan.getStart(), payloadEntrySpan.getSize()); + + if (expectedByteBuffer == null) { + Assert.assertEquals(StringUtils.format("expected empty buffer %s", index), EMPTY_BYTE_BUFFER, readByteBuffer); + } else { + Assert.assertEquals(StringUtils.format("failure on buffer %s", index), expectedByteBuffer, readByteBuffer); + } + } + + return masterByteBuffer; + } + + public ByteBuffer validateBufferWriteAndReadCell(List bufferList, boolean useRandomRead) throws IOException + { + ByteBuffer masterByteBuffer = writePayloadList(bufferList, new BufferWriterAsBytes()); + CellReader cellReader = new CellReader.Builder(masterByteBuffer) + .setCompressionStrategy(compressionStrategy) + .build(uncompressedBlock1, uncompressedBlock2); + + List positions = new ArrayList<>(bufferList.size()); + + for (int i = 0; i < bufferList.size(); i++) { + positions.add(i); + } + + Random random = new Random(0); + + if (useRandomRead) { + Collections.shuffle(positions, random); + } + + + for (int index : positions) { + ByteBuffer expectedByteBuffer = bufferList.get(index); + + ByteBuffer readByteBuffer = cellReader.getCell(index); + if (expectedByteBuffer == null) { + Assert.assertEquals(StringUtils.format("failure on buffer %s", index), 0L, readByteBuffer.remaining()); + } else { + Assert.assertEquals(StringUtils.format("failure on buffer %s", index), expectedByteBuffer, readByteBuffer); + } + } + + return masterByteBuffer; + } + + public ByteWriterTestHelper setUseRandomReadOrder(boolean useReadRandom) + { + validationFunctionBuilder.setReadRandom(useReadRandom); + + return this; + } + + public interface BufferWriter + { + void writeTo(BytesWriter writer, ByteBuffer payload) throws IOException; + } + + public static class BufferWriterAsBytes implements BufferWriter + { + public static final BufferWriterAsBytes INSTANCE = new BufferWriterAsBytes(); + + @Override + public void writeTo(BytesWriter writer, ByteBuffer payload) throws IOException + { + if (payload == null) { + writer.write((byte[]) null); + } else { + writer.write(payload.array()); + } + } + } + + public static class BufferWriterAsBuffer implements BufferWriter + { + public static final BufferWriterAsBuffer INSTANCE = new BufferWriterAsBuffer(); + + @Override + public void writeTo(BytesWriter writer, ByteBuffer payload) throws IOException + { + writer.write(payload); + } + } + + public interface ValidationFunction + { + ByteBuffer validateBufferList(List byteBufferList) throws Exception; + } + + public interface ValidationFunctionBuilder + { + ValidationFunctionBuilder PAYLOAD_WRITER_VALIDATION_FUNCTION_FACTORY = new PayloadWriterValidationFunctionBuilder(); + + ValidationFunctionBuilder ROW_READER_VALIDATION_FUNCTION_FACTORY = new RowReaderValidationFunctionBuilder(); + + ValidationFunction create(ByteWriterTestHelper testHelper); + + ValidationFunctionBuilder setReadRandom(boolean useRandomRead); + } + + public static class PayloadWriterValidationFunctionBuilder implements ValidationFunctionBuilder + { + private boolean useRandomRead; + + @Override + public ValidationFunction create(ByteWriterTestHelper testHelper) + { + return bufferList -> testHelper.validateBufferWriteAndReadBlockCompressed(bufferList, useRandomRead); + } + + @Override + public ValidationFunctionBuilder setReadRandom(boolean useRandomRead) + { + this.useRandomRead = useRandomRead; + return this; + } + } + + public static class RowReaderValidationFunctionBuilder implements ValidationFunctionBuilder + { + private boolean useRandomRead; + + @Override + public ValidationFunction create(ByteWriterTestHelper testHelper) + { + return bufferList -> testHelper.validateBufferWriteAndReadCell(bufferList, useRandomRead); + } + + @Override + public ValidationFunctionBuilder setReadRandom(boolean useRandomRead) + { + this.useRandomRead = useRandomRead; + return this; + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTest.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTest.java new file mode 100644 index 000000000000..921c3f22a0e2 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTest.java @@ -0,0 +1,47 @@ +/* + * 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.segment.serde.cell; + +public interface BytesReadWriteTest +{ + void testSingleWriteBytes() throws Exception; + + void testSingleMultiBlockWriteBytes() throws Exception; + + void testSingleMultiBlockWriteBytesWithPrelude() throws Exception; + + void testEmptyByteArray() throws Exception; + + void testNull() throws Exception; + + void testSingleLong() throws Exception; + + void testVariableSizedCompressablePayloads() throws Exception; + + void testOutliersInNormalDataUncompressablePayloads() throws Exception; + + void testOutliersInNormalDataCompressablePayloads() throws Exception; + + void testSingleUncompressableBlock() throws Exception; + + void testSingleWriteByteBufferZSTD() throws Exception; + + void testRandomBlockAccess() throws Exception; +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestBase.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestBase.java new file mode 100644 index 000000000000..94770ad00534 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestBase.java @@ -0,0 +1,244 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.collections.ResourceHolder; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.segment.data.CompressionStrategy; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Test; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +// base class used only for extension +public abstract class BytesReadWriteTestBase implements BytesReadWriteTest +{ + protected final BytesWriterBuilder bytesWriterBuilder; + + private final TestCasesConfig testCases; + private final ByteWriterTestHelper.ValidationFunctionBuilder validationFunctionBuilder; + + private ByteWriterTestHelper testHelper; + private Closer closer; + + protected BytesReadWriteTestBase( + BytesWriterBuilder bytesWriterBuilder, + ByteWriterTestHelper.ValidationFunctionBuilder validationFunctionBuilder, + TestCasesConfig testCases + ) + { + this.testCases = testCases; + this.bytesWriterBuilder = bytesWriterBuilder; + this.validationFunctionBuilder = validationFunctionBuilder; + } + + @Before + public void setup() + { + ResourceHolder uncompressedBlockHolder1 = NativeClearedByteBufferProvider.INSTANCE.get(); + ResourceHolder uncompressedBlockHolder2 = NativeClearedByteBufferProvider.INSTANCE.get(); + + closer = Closer.create(); + closer.register(uncompressedBlockHolder1); + closer.register(uncompressedBlockHolder2); + testHelper = new ByteWriterTestHelper( + bytesWriterBuilder, + uncompressedBlockHolder1.get(), + uncompressedBlockHolder2.get(), + validationFunctionBuilder + ); + } + + @After + public void tearDown() throws Exception + { + closer.close(); + } + + @Test + @Override + public void testSingleWriteBytes() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + + ByteBuffer payload = testHelper.generateBufferWithLongs(1024); + + runTestWithExceptionHandling(Collections.singletonList(payload), testCases.currentTestValue()); + } + + @Test + @Override + public void testSingleMultiBlockWriteBytes() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + + ByteBuffer payload = testHelper.generateBufferWithLongs(256 * 1024); // 2mb + + runTestWithExceptionHandling(Collections.singletonList(payload), testCases.currentTestValue()); + } + + @Test + @Override + public void testSingleMultiBlockWriteBytesWithPrelude() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + ByteBuffer payload1 = testHelper.generateBufferWithLongs(1024); // 8 kb + ByteBuffer payload2 = testHelper.generateBufferWithLongs(256 * 1024); // 256kb * 8 = 2mb + + runTestWithExceptionHandling(Arrays.asList(payload1, payload2), testCases.currentTestValue()); + } + + @Test + @Override + public void testEmptyByteArray() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + // no-op + ByteBuffer payload = ByteBuffer.wrap(new byte[0]); + // block index size: "8" : 4 bytes + // block index entry 0: "0": 4 bytes + // block index entry 1: "1": 4 bytes + // data stream size : "0" : 4 bytes + runTestWithExceptionHandling(Collections.singletonList(payload), testCases.currentTestValue()); + } + + @Test + @Override + public void testNull() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + TestCaseResult testCaseResult = testCases.currentTestValue(); + + runTestWithExceptionHandling(Collections.singletonList(null), testCaseResult); + } + + @Test + @Override + public void testSingleLong() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + ByteBuffer payload = testHelper.generateBufferWithLongs(1); + // block index size: "8" : 4 bytes + // block index entry 0: "0": 4 bytes + // block index entry 1: "0": 4 bytes + // data stream size : "1" : 4 bytes + // compressed single 8 bytes: 9 bytes (compressed: "0") + runTestWithExceptionHandling(Collections.singletonList(payload), testCases.currentTestValue()); + } + + @Test + @Override + public void testVariableSizedCompressablePayloads() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + + List bufferList = testHelper.generateRaggedPayloadBuffer(100, 1024, 10, 0, 0, 10); + + runTestWithExceptionHandling(bufferList, testCases.currentTestValue()); + } + + @Test + @Override + public void testOutliersInNormalDataUncompressablePayloads() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + + // every integer within a payload is unique + List bufferList = testHelper.generateRaggedPayloadBuffer(100, 1024, 10, 64 * 1024, 2); + + runTestWithExceptionHandling(bufferList, testCases.currentTestValue()); + } + + @Test + @Override + public void testOutliersInNormalDataCompressablePayloads() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + + // same # of payloads and size of payloads as testOutliersInNormalDataUncompressablePayloads() + // integer values range 0-9 + List bufferList = testHelper.generateRaggedPayloadBuffer(100, 1024, 10, 64 * 1024, 2, 10); + + runTestWithExceptionHandling(bufferList, testCases.currentTestValue()); + } + + @Test + @Override + public void testSingleUncompressableBlock() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + + // every integer within a payload is unique + ByteBuffer byteBuffer = testHelper.generateIntPayloads(16 * 1024); + + Assert.assertEquals(64 * 1024, byteBuffer.limit()); + // uncompressable 64k block size + runTestWithExceptionHandling(Collections.singletonList(byteBuffer), testCases.currentTestValue()); + + } + + @Test + @Override + public void testSingleWriteByteBufferZSTD() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + + ByteBuffer sourcePayLoad = testHelper.generateBufferWithLongs(1024); // 8k + + testHelper.setCompressionStrategy(CompressionStrategy.ZSTD); + runTestWithExceptionHandling(Collections.singletonList(sourcePayLoad), testCases.currentTestValue()); + } + + @Test + @Override + public void testRandomBlockAccess() throws Exception + { + Assume.assumeTrue(testCases.isCurrentTestEnabled()); + //verified that blocks are accessed in random order and the same block is even returned to + List bufferList = testHelper.generateRaggedPayloadBuffer(8192, 32 * 1024, 256, 256 * 1024, 3, 1024); + + testHelper.setUseRandomReadOrder(true); + runTestWithExceptionHandling(bufferList, testCases.currentTestValue()); + } + + private void runTestWithExceptionHandling(List bufferList, TestCaseResult testCaseResult) throws Exception + { + try { + testHelper.validateReadAndSize(bufferList, testCaseResult.size); + + if (testCaseResult.exception != null) { + Assert.fail("expected exception " + testCaseResult.exception.getClass().getName()); + } + } + catch (Exception e) { + if (testCaseResult.exception != null) { + Assert.assertTrue(testCaseResult.exception.getClass().isAssignableFrom(e.getClass())); + } else { + throw e; + } + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestCases.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestCases.java new file mode 100644 index 000000000000..2bedcfb8217e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesReadWriteTestCases.java @@ -0,0 +1,28 @@ +/* + * 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.segment.serde.cell; + +public class BytesReadWriteTestCases extends TestCasesConfig +{ + public BytesReadWriteTestCases() + { + super(BytesReadWriteTest.class, BytesReadWriteTestBase.class); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriter.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriter.java new file mode 100644 index 000000000000..697beaa0ebe3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriter.java @@ -0,0 +1,43 @@ +/* + * 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.segment.serde.cell; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** + * this interface is used so that both CellWriter[.Builder] and BlockCompressedPayloadWriter[.Builder] may use the + * same test code. production code should not use this and use the classes directly + */ +public interface BytesWriter extends Closeable +{ + void write(byte[] cellBytes) throws IOException; + + void write(ByteBuffer rowByteBuffer) throws IOException; + + @Override + void close() throws IOException; + + void transferTo(WritableByteChannel channel) throws IOException; + + long getSerializedSize(); +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriterBuilder.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriterBuilder.java new file mode 100644 index 000000000000..1330a44ce3f7 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/BytesWriterBuilder.java @@ -0,0 +1,36 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.segment.data.CompressionStrategy; + +import java.io.IOException; + +/** + * this interface is used so that both RowWriter[.Builder] and BlockCompressedPayload[.Builder] may use the + * same test code. production code should not use this and use the classes directly + */ + +public interface BytesWriterBuilder +{ + BytesWriter build() throws IOException; + + BytesWriterBuilder setCompressionStrategy(CompressionStrategy compressionStrategy); +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterTest.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterTest.java new file mode 100644 index 000000000000..e375eb695157 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterTest.java @@ -0,0 +1,52 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; + +public class CellWriterTest extends BytesReadWriteTestBase +{ + public CellWriterTest() + { + super( + new CellWriterToBytesWriter.Builder( + new CellWriter.Builder(NativeClearedByteBufferProvider.INSTANCE, new OnHeapMemorySegmentWriteOutMedium()) + ), + ByteWriterTestHelper.ValidationFunctionBuilder.ROW_READER_VALIDATION_FUNCTION_FACTORY, + new BytesReadWriteTestCases() + .setTestCaseValue(BytesReadWriteTest::testSingleLong, TestCaseResult.of(62)) + .setTestCaseValue(BytesReadWriteTest::testEmptyByteArray, TestCaseResult.of(46)) + .setTestCaseValue(BytesReadWriteTest::testNull, TestCaseResult.of(46)) + .setTestCaseValue(BytesReadWriteTest::testSingleWriteBytes, TestCaseResult.of(4151)) + .setTestCaseValue(BytesReadWriteTest::testSingleMultiBlockWriteBytes, TestCaseResult.of(1049204)) + .setTestCaseValue(BytesReadWriteTest::testSingleMultiBlockWriteBytesWithPrelude, TestCaseResult.of(1053277)) + .setTestCaseValue(BytesReadWriteTest::testVariableSizedCompressablePayloads, TestCaseResult.of(1655)) + .setTestCaseValue(BytesReadWriteTest::testOutliersInNormalDataCompressablePayloads, TestCaseResult.of(7368)) + .setTestCaseValue( + BytesReadWriteTest::testOutliersInNormalDataUncompressablePayloads, + TestCaseResult.of(575673) + ) + .setTestCaseValue(BytesReadWriteTest::testSingleUncompressableBlock, TestCaseResult.of(65750)) + .setTestCaseValue(BytesReadWriteTest::testSingleWriteByteBufferZSTD, TestCaseResult.of(845)) + .setTestCaseValue(BytesReadWriteTest::testRandomBlockAccess, TestCaseResult.of(3126618)) + + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterToBytesWriter.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterToBytesWriter.java new file mode 100644 index 000000000000..a159a6a77c8c --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/CellWriterToBytesWriter.java @@ -0,0 +1,90 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.segment.data.CompressionStrategy; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +public class CellWriterToBytesWriter implements BytesWriter +{ + private final CellWriter cellWriter; + + public CellWriterToBytesWriter(CellWriter cellWriter) + { + this.cellWriter = cellWriter; + } + + @Override + public void write(byte[] cellBytes) throws IOException + { + cellWriter.write(cellBytes); + } + + @Override + public void write(ByteBuffer rowByteBuffer) throws IOException + { + cellWriter.write(rowByteBuffer); + } + + @Override + public void transferTo(WritableByteChannel channel) throws IOException + { + cellWriter.writeTo(channel, null); + } + + @Override + public void close() throws IOException + { + cellWriter.close(); + } + + @Override + public long getSerializedSize() + { + return cellWriter.getSerializedSize(); + } + + public static class Builder implements BytesWriterBuilder + { + private final CellWriter.Builder builder; + + public Builder(CellWriter.Builder builder) + { + this.builder = builder; + } + + @Override + public BytesWriterBuilder setCompressionStrategy(CompressionStrategy compressionStrategy) + { + builder.setCompressionStrategy(compressionStrategy); + + return this; + } + + @Override + public BytesWriter build() throws IOException + { + return new CellWriterToBytesWriter(builder.build()); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/CommitTestSuite.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/CommitTestSuite.java new file mode 100644 index 000000000000..3c296a652c6f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/CommitTestSuite.java @@ -0,0 +1,44 @@ +/* + * 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.segment.serde.cell; + +import org.apache.druid.query.aggregation.SerializablePairLongStringBufferStoreTest; +import org.apache.druid.query.aggregation.SerializablePairLongStringComplexMetricSerdeTest; +import org.apache.druid.query.aggregation.SerializablePairLongStringDeltaEncodedStagedSerdeTest; +import org.apache.druid.query.aggregation.SerializablePairLongStringSimpleStagedSerdeTest; +import org.apache.druid.query.aggregation.first.StringFirstTimeseriesQueryTest; +import org.apache.druid.query.aggregation.last.StringLastTimeseriesQueryTest; +import org.junit.runner.RunWith; +import org.junit.runners.Suite; + +@RunWith(Suite.class) +@Suite.SuiteClasses({ + StringFirstTimeseriesQueryTest.class, + StringLastTimeseriesQueryTest.class, + SerializablePairLongStringBufferStoreTest.class, + SerializablePairLongStringComplexMetricSerdeTest.class, + SerializablePairLongStringDeltaEncodedStagedSerdeTest.class, + SerializablePairLongStringSimpleStagedSerdeTest.class, + BlockCompressedPayloadWriterTest.class, + CellWriterTest.class +}) +public class CommitTestSuite +{ +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/RandomStringUtils.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/RandomStringUtils.java new file mode 100644 index 000000000000..a7b148fcdac9 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/RandomStringUtils.java @@ -0,0 +1,49 @@ +/* + * 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.segment.serde.cell; + +import java.util.Random; + +public class RandomStringUtils +{ + private final Random random; + + public RandomStringUtils() + { + random = new Random(0); + } + + public RandomStringUtils(Random random) + { + this.random = random; + } + + public String randomAlphanumeric(int length) + { + return org.apache.commons.lang3.RandomStringUtils.random(length, 0, 0, true, true, null, random); + } + + public String randomAlphanumeric(int minLength, int maxLength) + { + int length = random.nextInt(maxLength - minLength) + minLength; + + return org.apache.commons.lang3.RandomStringUtils.random(length, 0, 0, true, true, null, random); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/TestCaseResult.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/TestCaseResult.java new file mode 100644 index 000000000000..a75b29fd0ec6 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/TestCaseResult.java @@ -0,0 +1,54 @@ +/* + * 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.segment.serde.cell; + +public class TestCaseResult +{ + public final byte[] bytes; + public final int size; + public final Exception exception; + + private TestCaseResult(byte[] bytes, int size, Exception exception) + { + this.bytes = bytes; + this.size = size; + this.exception = exception; + } + + public static TestCaseResult of(Exception exception) + { + return new TestCaseResult(null, -1, exception); + } + + public static TestCaseResult of(int sizeBytes) + { + return new TestCaseResult(null, sizeBytes, null); + } + + public static TestCaseResult of(byte[] bytes) + { + return new TestCaseResult(bytes, bytes.length, null); + } + + public static TestCaseResult of(byte[] bytes, int sizeBytes) + { + return new TestCaseResult(bytes, sizeBytes, null); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/serde/cell/TestCasesConfig.java b/processing/src/test/java/org/apache/druid/segment/serde/cell/TestCasesConfig.java new file mode 100644 index 000000000000..078113cc72a1 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/serde/cell/TestCasesConfig.java @@ -0,0 +1,199 @@ +/* + * 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.segment.serde.cell; + +import java.lang.invoke.MethodHandles; +import java.lang.invoke.MethodType; +import java.lang.reflect.InvocationHandler; +import java.lang.reflect.Method; +import java.lang.reflect.Proxy; +import java.util.LinkedHashMap; +import java.util.Map; + +public class TestCasesConfig +{ + private final MethodCallCapturer methodCallCapturer; + private final Class testCasesInterface; + private final Class testClassImpl; + private final Map testCasesToRun = new LinkedHashMap<>(); + + public TestCasesConfig(Class testCasesInterface, Class testClassImpl) + { + methodCallCapturer = new MethodCallCapturer<>(testCasesInterface); + this.testCasesInterface = testCasesInterface; + this.testClassImpl = testClassImpl; + } + + public TestCasesConfig setTestCaseValue(TestMethodHandle testMethodHandle, TestCaseResult expectedResult) + { + testCasesToRun.put(testMethodHandle, expectedResult); + + return this; + } + + public TestCasesConfig setTestCaseValue(MethodAccess methodAccess, TestCaseResult expectedResult) + { + TestMethodHandle testMethodHandle = capture(methodAccess); + testCasesToRun.put(testMethodHandle, expectedResult); + + return this; + } + + public TestCasesConfig setTestCaseValue(MethodAccess methodAccess, int sizeBytes) + { + TestMethodHandle testMethodHandle = capture(methodAccess); + testCasesToRun.put(testMethodHandle, TestCaseResult.of(sizeBytes)); + + return this; + } + + public TestCasesConfig setTestCaseValue(MethodAccess methodAccess, byte[] bytes) + { + TestMethodHandle testMethodHandle = capture(methodAccess); + testCasesToRun.put(testMethodHandle, TestCaseResult.of(bytes)); + + return this; + } + + public TestCasesConfig enableTestCase(MethodAccess methodAccess) + { + TestMethodHandle testMethodHandle = capture(methodAccess); + testCasesToRun.put(testMethodHandle, TestCaseResult.of(-1)); + + return this; + } + + public TestCaseResult currentTestValue() + { + TestMethodHandle currentTestMethodHandle = getCurrentTestMethod(); + return testCasesToRun.get(currentTestMethodHandle); + } + + public boolean isCurrentTestEnabled() + { + TestMethodHandle currentTestMethodHandle = getCurrentTestMethod(); + return testCasesToRun.containsKey(currentTestMethodHandle); + } + + private TestMethodHandle capture(MethodAccess access) + { + try { + Method method = methodCallCapturer.captureMethod(access); + TestMethodHandle testMethodHandle = new TestMethodHandle(method.getName()); + + return testMethodHandle; + } + catch (Throwable e) { + throw new RuntimeException(e); + } + } + + private TestMethodHandle getCurrentTestMethod() + { + StackTraceElement[] stackTrace = Thread.currentThread().getStackTrace(); + String thisMethodName = stackTrace[3].getMethodName(); + + return new TestMethodHandle(thisMethodName); + } + + public class TestMethodHandle + { + private final String name; + + public TestMethodHandle(String name) + { + this.name = name; + try { + // validate method exists + MethodHandles.lookup() + .findVirtual(testCasesInterface, name, MethodType.methodType(void.class)); + // validate method exists + MethodHandles.lookup() + .findVirtual(testClassImpl, name, MethodType.methodType(void.class)); + } + catch (NoSuchMethodException | IllegalAccessException e) { + throw new RuntimeException(e); + } + } + + public String getName() + { + return testCasesInterface.getName() + "::void " + name + "()"; + } + + @Override + public int hashCode() + { + return getName().hashCode(); + } + + @Override + public boolean equals(Object obj) + { + if (obj != null && this.getClass().equals(obj.getClass())) { + return getName().equals(((TestMethodHandle) obj).getName()); + } + + return false; + } + + + @Override + public String toString() + { + return getName(); + } + } + + public interface MethodAccess + { + void access(I input) throws T; + } + + private static class MethodCallCapturer implements InvocationHandler + { + private volatile Method lastMethod = null; + private final T wrapper; + + @SuppressWarnings("unchecked") + public MethodCallCapturer(Class clazz) + { + wrapper = (T) Proxy.newProxyInstance(clazz.getClassLoader(), new Class[]{clazz}, this); + } + + public Method captureMethod(MethodAccess access) throws Throwable + { + access.access(wrapper); + + return lastMethod; + } + + + @SuppressWarnings("ReturnOfNull") + @Override + public Object invoke(Object proxy, Method method, Object[] args) + { + lastMethod = method; + + // unused + return null; + } + } +}