Skip to content

Commit

Permalink
Enable BYTE_STREAM_SPLIT for parquet
Browse files Browse the repository at this point in the history
Introduced DoubleApacheParquetValueDecoder and FloatApacheParquetValueDecoder to handle double and float respectively.
Added a unit test for Double.
  • Loading branch information
manupatteri committed Feb 29, 2024
1 parent 6f9f8cf commit e43da5b
Show file tree
Hide file tree
Showing 9 changed files with 374 additions and 22 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.values.ValuesReader;
import org.apache.parquet.column.values.bitpacking.ByteBitPackingValuesReader;
import org.apache.parquet.column.values.bytestreamsplit.ByteStreamSplitValuesReaderForDouble;
import org.apache.parquet.column.values.bytestreamsplit.ByteStreamSplitValuesReaderForFloat;
import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader;
import org.apache.parquet.column.values.deltalengthbytearray.DeltaLengthByteArrayValuesReader;
import org.apache.parquet.column.values.deltastrings.DeltaByteArrayReader;
Expand All @@ -44,7 +46,9 @@
import static org.apache.parquet.column.values.bitpacking.Packer.BIG_ENDIAN;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;

Expand Down Expand Up @@ -149,6 +153,20 @@ public Dictionary initDictionary(ColumnDescriptor descriptor, DictionaryPage dic
{
return PLAIN.initDictionary(descriptor, dictionaryPage);
}
},

BYTE_STREAM_SPLIT {
@Override
public ValuesReader getValuesReader(ColumnDescriptor descriptor, ValuesType valuesType)
{
PrimitiveTypeName typeName = descriptor.getPrimitiveType().getPrimitiveTypeName();
checkArgument(typeName == FLOAT || typeName == DOUBLE, "Encoding BYTE_STREAM_SPLIT is only " +
"supported for type FLOAT and DOUBLE");
if (typeName == FLOAT) {
return new ByteStreamSplitValuesReaderForFloat();
}
return new ByteStreamSplitValuesReaderForDouble();
}
};

static final int INT96_TYPE_LENGTH = 12;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,28 +156,18 @@ private static int getPathIndex(List<PrimitiveColumnIO> columns, List<String> pa
@SuppressWarnings("deprecation")
public static ParquetEncoding getParquetEncoding(Encoding encoding)
{
switch (encoding) {
case PLAIN:
return ParquetEncoding.PLAIN;
case RLE:
return ParquetEncoding.RLE;
case BYTE_STREAM_SPLIT:
// TODO: https://github.com/trinodb/trino/issues/8357
throw new ParquetDecodingException("Unsupported Parquet encoding: " + encoding);
case BIT_PACKED:
return ParquetEncoding.BIT_PACKED;
case PLAIN_DICTIONARY:
return ParquetEncoding.PLAIN_DICTIONARY;
case DELTA_BINARY_PACKED:
return ParquetEncoding.DELTA_BINARY_PACKED;
case DELTA_LENGTH_BYTE_ARRAY:
return ParquetEncoding.DELTA_LENGTH_BYTE_ARRAY;
case DELTA_BYTE_ARRAY:
return ParquetEncoding.DELTA_BYTE_ARRAY;
case RLE_DICTIONARY:
return ParquetEncoding.RLE_DICTIONARY;
}
throw new ParquetDecodingException("Unsupported Parquet encoding: " + encoding);
return switch (encoding) {
case PLAIN -> ParquetEncoding.PLAIN;
case RLE -> ParquetEncoding.RLE;
case BYTE_STREAM_SPLIT -> ParquetEncoding.BYTE_STREAM_SPLIT;
case BIT_PACKED -> ParquetEncoding.BIT_PACKED;
case PLAIN_DICTIONARY -> ParquetEncoding.PLAIN_DICTIONARY;
case DELTA_BINARY_PACKED -> ParquetEncoding.DELTA_BINARY_PACKED;
case DELTA_LENGTH_BYTE_ARRAY -> ParquetEncoding.DELTA_LENGTH_BYTE_ARRAY;
case DELTA_BYTE_ARRAY -> ParquetEncoding.DELTA_BYTE_ARRAY;
case RLE_DICTIONARY -> ParquetEncoding.RLE_DICTIONARY;
default -> throw new ParquetDecodingException("Unsupported Parquet encoding: " + encoding);
};
}

public static org.apache.parquet.schema.Type getParquetTypeByName(String columnName, GroupType groupType)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@
import java.nio.ByteBuffer;

import static io.trino.parquet.ParquetReaderUtils.castToByte;
import static java.lang.Double.doubleToLongBits;
import static java.lang.Float.floatToIntBits;
import static java.util.Objects.requireNonNull;

/**
Expand Down Expand Up @@ -70,4 +72,91 @@ public void skip(int n)
delegate.skip(n);
}
}

public static final class DoubleApacheParquetValueDecoder
implements ValueDecoder<long[]>
{
private final ValuesReader delegate;

public DoubleApacheParquetValueDecoder(ValuesReader delegate)
{
this.delegate = requireNonNull(delegate, "delegate is null");
}

@Override
public void init(SimpleSliceInputStream input)
{
byte[] buffer = input.readBytes();
try {
int valueCount = buffer.length / Byte.SIZE;
delegate.initFromPage(valueCount, ByteBufferInputStream.wrap(ByteBuffer.wrap(buffer, 0, buffer.length)));
}
catch (IOException e) {
throw new UncheckedIOException(e);
}
}

@Override
public void read(long[] values, int offset, int length)
{
for (int i = offset; i < offset + length; i++) {
values[i] = doubleToLongBits(delegate.readDouble());
}
}

@Override
public void skip(int n)
{
delegate.skip(n);
}
}

public static final class FloatApacheParquetValueDecoder
implements ValueDecoder<int[]>
{
private final ValuesReader delegate;

public FloatApacheParquetValueDecoder(ValuesReader delegate)
{
this.delegate = requireNonNull(delegate, "delegate is null");
}

@Override
public void init(SimpleSliceInputStream input)
{
byte[] buffer = input.readBytes();
try {
int valueCount = buffer.length / Float.BYTES;
delegate.initFromPage(valueCount, ByteBufferInputStream.wrap(ByteBuffer.wrap(buffer, 0, buffer.length)));
}
catch (IOException e) {
throw new UncheckedIOException(e);
}
}

@Override
public void read(int[] values, int offset, int length)
{
for (int i = offset; i < offset + length; i++) {
values[i] = floatToIntBits(delegate.readFloat());
}
}

@Override
public void skip(int n)
{
delegate.skip(n);
}
}

private static void initialize(SimpleSliceInputStream input, ValuesReader reader)
{
byte[] buffer = input.readBytes();
try {
reader.initFromPage(0, ByteBufferInputStream.wrap(ByteBuffer.wrap(buffer, 0, buffer.length)));
}
catch (IOException e) {
throw new UncheckedIOException(e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.joda.time.DateTimeZone;

import static com.google.common.base.Preconditions.checkArgument;
import static io.trino.parquet.ParquetEncoding.BYTE_STREAM_SPLIT;
import static io.trino.parquet.ParquetEncoding.DELTA_BYTE_ARRAY;
import static io.trino.parquet.ParquetEncoding.PLAIN;
import static io.trino.parquet.ParquetReaderUtils.toByteExact;
Expand All @@ -45,6 +46,8 @@
import static io.trino.parquet.ParquetTypeUtils.getShortDecimalValue;
import static io.trino.parquet.ValuesType.VALUES;
import static io.trino.parquet.reader.decoders.ApacheParquetValueDecoders.BooleanApacheParquetValueDecoder;
import static io.trino.parquet.reader.decoders.ApacheParquetValueDecoders.DoubleApacheParquetValueDecoder;
import static io.trino.parquet.reader.decoders.ApacheParquetValueDecoders.FloatApacheParquetValueDecoder;
import static io.trino.parquet.reader.decoders.DeltaBinaryPackedDecoders.DeltaBinaryPackedByteDecoder;
import static io.trino.parquet.reader.decoders.DeltaBinaryPackedDecoders.DeltaBinaryPackedIntDecoder;
import static io.trino.parquet.reader.decoders.DeltaBinaryPackedDecoders.DeltaBinaryPackedLongDecoder;
Expand Down Expand Up @@ -112,6 +115,9 @@ public ValueDecoder<long[]> getDoubleDecoder(ParquetEncoding encoding)
if (PLAIN.equals(encoding)) {
return new LongPlainValueDecoder();
}
else if (BYTE_STREAM_SPLIT.equals(encoding)) {
return new DoubleApacheParquetValueDecoder(getApacheParquetReader(encoding));
}
throw wrongEncoding(encoding);
}

Expand All @@ -120,6 +126,9 @@ public ValueDecoder<int[]> getRealDecoder(ParquetEncoding encoding)
if (PLAIN.equals(encoding)) {
return new IntPlainValueDecoder();
}
else if (BYTE_STREAM_SPLIT.equals(encoding)) {
return new FloatApacheParquetValueDecoder(getApacheParquetReader(encoding));
}
throw wrongEncoding(encoding);
}

Expand Down
Loading

0 comments on commit e43da5b

Please sign in to comment.