Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Support reading BYTE_STREAM_SPLIT encoding in parquet #12809

Merged
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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 {
raunaqmorarka marked this conversation as resolved.
Show resolved Hide resolved
@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 @@ -27,7 +27,6 @@
import org.apache.parquet.io.ColumnIOFactory;
import org.apache.parquet.io.GroupColumnIO;
import org.apache.parquet.io.MessageColumnIO;
import org.apache.parquet.io.ParquetDecodingException;
import org.apache.parquet.io.PrimitiveColumnIO;
import org.apache.parquet.schema.GroupType;
import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation;
Expand Down Expand Up @@ -156,28 +155,17 @@ 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) {
raunaqmorarka marked this conversation as resolved.
Show resolved Hide resolved
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;
};
}

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,78 @@ public void skip(int n)
delegate.skip(n);
}
}

public static final class DoubleApacheParquetValueDecoder
raunaqmorarka marked this conversation as resolved.
Show resolved Hide resolved
implements ValueDecoder<long[]>
{
private final ValuesReader delegate;

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

@Override
public void init(SimpleSliceInputStream input)
{
initialize(input, delegate, Double.BYTES);
}

@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
raunaqmorarka marked this conversation as resolved.
Show resolved Hide resolved
implements ValueDecoder<int[]>
{
private final ValuesReader delegate;

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

@Override
public void init(SimpleSliceInputStream input)
{
initialize(input, delegate, Float.BYTES);
}

@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, int elementSizeInBytes)
{
byte[] buffer = input.readBytes();
try {
int valueCount = buffer.length / elementSizeInBytes;
reader.initFromPage(valueCount, 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
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
/*
* Licensed 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 io.trino.parquet.reader;

import com.google.common.collect.ImmutableList;
import com.google.common.io.Resources;
import io.trino.parquet.ParquetDataSource;
import io.trino.parquet.ParquetReaderOptions;
import io.trino.spi.Page;
import io.trino.spi.block.Block;
import io.trino.spi.block.IntArrayBlock;
import io.trino.spi.type.Type;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.testng.annotations.Test;

import java.io.File;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.List;
import java.util.Optional;
import java.util.stream.IntStream;

import static io.trino.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext;
import static io.trino.parquet.ParquetTestUtils.createParquetReader;
import static io.trino.spi.type.DoubleType.DOUBLE;
import static io.trino.spi.type.RealType.REAL;
import static io.trino.testing.TestingConnectorSession.SESSION;
import static org.assertj.core.api.Assertions.assertThat;
import static org.testng.Assert.assertEquals;

public class TestByteStreamSplitEncoding
{
@Test
public void testReadFloatDouble()
throws URISyntaxException, IOException
{
List<String> columnNames = ImmutableList.of("columnA", "columnB");
List<Type> types = ImmutableList.of(REAL, DOUBLE);

ParquetDataSource dataSource = new FileParquetDataSource(
new File(Resources.getResource("byte_stream_split_float_and_double.parquet").toURI()),
raunaqmorarka marked this conversation as resolved.
Show resolved Hide resolved
new ParquetReaderOptions());
ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty());
ParquetReader reader = createParquetReader(dataSource, parquetMetadata, newSimpleAggregatedMemoryContext(), types, columnNames);

readAndCompare(reader, getExpectedValues());
}

private static List<List<Double>> getExpectedValues()
{
ImmutableList.Builder<Double> floatsBuilder = ImmutableList.builder();
for (int i = 0; i < 10; i++) {
IntStream.range(0, 10)
.mapToDouble(j -> j * 1.3)
.forEach(floatsBuilder::add);
}

ImmutableList.Builder<Double> doublesBuilder = ImmutableList.builder();
for (int i = 0; i < 10; i++) {
IntStream.range(0, 10)
.mapToDouble(j -> j * 1.5)
.forEach(doublesBuilder::add);
}
return ImmutableList.of(floatsBuilder.build(), doublesBuilder.build());
}

private static void readAndCompare(ParquetReader reader, List<List<Double>> expected)
throws IOException
{
int rowCount = 0;
int pageCount = 0;
Page page = reader.nextPage();
while (page != null) {
assertThat(page.getChannelCount()).isEqualTo(2);
if (pageCount % 2 == 1) { // Skip loading every alternative page
for (int channel = 0; channel < page.getChannelCount(); channel++) {
Block block = page.getBlock(channel).getLoadedBlock();
List<Double> expectedValues = expected.get(channel);
for (int postition = 0; postition < block.getPositionCount(); postition++) {
if (block instanceof IntArrayBlock) {
assertEquals(REAL.getObjectValue(SESSION, block, postition), expectedValues.get(rowCount + postition).floatValue());
}
else {
assertEquals(DOUBLE.getObjectValue(SESSION, block, postition), expectedValues.get(rowCount + postition));
}
}
}
}
rowCount += page.getPositionCount();
pageCount++;
page = reader.nextPage();
}
assertThat(rowCount).isEqualTo(100);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,18 +15,16 @@

import com.google.common.collect.ImmutableList;
import io.trino.parquet.PrimitiveField;
import io.trino.parquet.reader.SimpleSliceInputStream;
import io.trino.spi.type.DoubleType;
import org.apache.parquet.column.values.ValuesReader;
import org.apache.parquet.column.values.ValuesWriter;

import java.util.OptionalInt;
import java.util.Random;

import static io.trino.parquet.ParquetEncoding.PLAIN;
import static io.trino.parquet.ParquetEncoding.RLE_DICTIONARY;
import static io.trino.parquet.reader.decoders.ApacheParquetValueDecoders.DoubleApacheParquetValueDecoder;
import static io.trino.parquet.reader.flat.LongColumnAdapter.LONG_ADAPTER;
import static java.util.Objects.requireNonNull;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
import static org.assertj.core.api.Assertions.assertThat;

Expand Down Expand Up @@ -108,35 +106,4 @@ private static DataBuffer writeDoubles(ValuesWriter valuesWriter, double[] input

return getWrittenBuffer(valuesWriter);
}

private 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)
{
initialize(input, delegate);
}

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

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