diff --git a/LICENSE b/LICENSE index 34511297cd8a..80cfd3652e69 100644 --- a/LICENSE +++ b/LICENSE @@ -288,6 +288,7 @@ This product includes code from Apache Spark. * casting logic in AssignmentAlignmentSupport * implementation of SetAccumulator. * Connector expressions. +* implementation of VectorizedDeltaEncodedValuesReader Copyright: 2011-2018 The Apache Software Foundation Home page: https://spark.apache.org/ diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDeltaEncodedValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDeltaEncodedValuesReader.java new file mode 100644 index 000000000000..115518e1fb50 --- /dev/null +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDeltaEncodedValuesReader.java @@ -0,0 +1,283 @@ +/* + * 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.iceberg.arrow.vectorized.parquet; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import org.apache.arrow.vector.FieldVector; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.parquet.bytes.ByteBufferInputStream; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.column.values.ValuesReader; +import org.apache.parquet.column.values.bitpacking.BytePackerForLong; +import org.apache.parquet.column.values.bitpacking.Packer; +import org.apache.parquet.io.ParquetDecodingException; +import org.apache.parquet.io.api.Binary; + +/** + * A {@link VectorizedValuesReader} implementation for the encoding type DELTA_BINARY_PACKED. This + * is adapted from Spark's VectorizedDeltaBinaryPackedReader. + * + * @see + * Parquet format encodings: DELTA_BINARY_PACKED + */ +public class VectorizedDeltaEncodedValuesReader extends ValuesReader + implements VectorizedValuesReader { + + // header data + private int blockSizeInValues; + private int miniBlocksPerBlock; + private int totalValueCount; + private long firstValue; + + private int miniBlockSizeInValues; + + // values read by the caller + private int valuesRead = 0; + + // variables to keep state of the current block and miniblock + private long lastValueRead; // needed to compute the next value + private long minDeltaInCurrentBlock; // needed to compute the next value + // currentMiniBlock keeps track of the mini block within the current block that + // we read and decoded most recently. Only used as an index into + // bitWidths array + private int currentMiniBlock = 0; + private int[] bitWidths; // bit widths for each miniBlock in the current block + private int remainingInBlock = 0; // values in current block still to be read + private int remainingInMiniBlock = 0; // values in current mini block still to be read + private long[] unpackedValuesBuffer; + + private ByteBufferInputStream inputStream; + + // temporary buffers used by readInteger and readLong + private int intVal; + private long longVal; + + @Override + public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException { + Preconditions.checkArgument( + valueCount >= 1, "Page must have at least one value, but it has " + valueCount); + this.inputStream = in; + // Read the header + this.blockSizeInValues = BytesUtils.readUnsignedVarInt(this.inputStream); + this.miniBlocksPerBlock = BytesUtils.readUnsignedVarInt(this.inputStream); + double miniSize = (double) blockSizeInValues / miniBlocksPerBlock; + Preconditions.checkArgument( + miniSize % 8 == 0, "miniBlockSize must be multiple of 8, but it's " + miniSize); + this.miniBlockSizeInValues = (int) miniSize; + // True value count. May be less than valueCount because of nulls + this.totalValueCount = BytesUtils.readUnsignedVarInt(this.inputStream); + this.bitWidths = new int[miniBlocksPerBlock]; + this.unpackedValuesBuffer = new long[miniBlockSizeInValues]; + // read the first value + firstValue = BytesUtils.readZigZagVarLong(this.inputStream); + } + + /** DELTA_BINARY_PACKED only supports INT32 and INT64 */ + @Override + public byte readByte() { + throw new UnsupportedOperationException("readByte is not supported"); + } + + /** DELTA_BINARY_PACKED only supports INT32 and INT64 */ + @Override + public short readShort() { + throw new UnsupportedOperationException("readShort is not supported"); + } + + @Override + public int readInteger() { + readValues(1, null, 0, INT_SIZE, (f, i, v) -> intVal = (int) v); + return intVal; + } + + @Override + public long readLong() { + readValues(1, null, 0, LONG_SIZE, (f, i, v) -> longVal = v); + return longVal; + } + + /** The Iceberg reader currently does not do skipping */ + @Override + public void skip() { + throw new UnsupportedOperationException("skip is not supported"); + } + + /** DELTA_BINARY_PACKED only supports INT32 and INT64 */ + @Override + public Binary readBinary(int len) { + throw new UnsupportedOperationException("readBinary is not supported"); + } + + @Override + public void readIntegers(int total, FieldVector vec, int rowId) { + readValues(total, vec, rowId, INT_SIZE, (f, i, v) -> f.getDataBuffer().setInt(i, (int) v)); + } + + @Override + public void readLongs(int total, FieldVector vec, int rowId) { + readValues(total, vec, rowId, LONG_SIZE, (f, i, v) -> f.getDataBuffer().setLong(i, v)); + } + + /** DELTA_BINARY_PACKED only supports INT32 and INT64 */ + @Override + public void readFloats(int total, FieldVector vec, int rowId) { + throw new UnsupportedOperationException("readFloats is not supported"); + } + + /** DELTA_BINARY_PACKED only supports INT32 and INT64 */ + @Override + public void readDoubles(int total, FieldVector vec, int rowId) { + throw new UnsupportedOperationException("readDoubles is not supported"); + } + + private void readValues( + int total, FieldVector vec, int rowId, int typeWidth, IntegerOutputWriter outputWriter) { + if (valuesRead + total > totalValueCount) { + throw new ParquetDecodingException( + "No more values to read. Total values read: " + + valuesRead + + ", total count: " + + totalValueCount + + ", trying to read " + + total + + " more."); + } + + int remaining = total; + int currentRowId = rowId; + // First value + if (valuesRead == 0) { + outputWriter.write(vec, ((long) (currentRowId + valuesRead) * typeWidth), firstValue); + lastValueRead = firstValue; + currentRowId++; + remaining--; + } + + while (remaining > 0) { + int loadedRows; + try { + loadedRows = loadMiniBlockToOutput(remaining, vec, currentRowId, typeWidth, outputWriter); + } catch (IOException e) { + throw new ParquetDecodingException("Error reading mini block.", e); + } + currentRowId += loadedRows; + remaining -= loadedRows; + } + valuesRead = total - remaining; + } + + /** + * Read from a mini block. Read at most 'remaining' values into output. + * + * @return the number of values read into output + */ + private int loadMiniBlockToOutput( + int remaining, FieldVector vec, int rowId, int typeWidth, IntegerOutputWriter outputWriter) + throws IOException { + + // new block; read the block header + if (remainingInBlock == 0) { + readBlockHeader(); + } + + // new miniblock, unpack the miniblock + if (remainingInMiniBlock == 0) { + unpackMiniBlock(); + } + + // read values from miniblock + int valuesReadInMiniBlock = 0; + for (int i = miniBlockSizeInValues - remainingInMiniBlock; + i < miniBlockSizeInValues && valuesReadInMiniBlock < remaining; + i++) { + // calculate values from deltas unpacked for current block + long outValue = lastValueRead + minDeltaInCurrentBlock + unpackedValuesBuffer[i]; + lastValueRead = outValue; + outputWriter.write(vec, ((long) (rowId + valuesReadInMiniBlock) * typeWidth), outValue); + remainingInBlock--; + remainingInMiniBlock--; + valuesReadInMiniBlock++; + } + + return valuesReadInMiniBlock; + } + + private void readBlockHeader() { + try { + minDeltaInCurrentBlock = BytesUtils.readZigZagVarLong(inputStream); + } catch (IOException e) { + throw new ParquetDecodingException("Can not read min delta in current block", e); + } + readBitWidthsForMiniBlocks(); + remainingInBlock = blockSizeInValues; + currentMiniBlock = 0; + remainingInMiniBlock = 0; + } + + /** + * mini block has a size of 8*n, unpack 32 value each time + * + *
see org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader#unpackMiniBlock + */ + private void unpackMiniBlock() throws IOException { + Arrays.fill(this.unpackedValuesBuffer, 0); + BytePackerForLong packer = + Packer.LITTLE_ENDIAN.newBytePackerForLong(bitWidths[currentMiniBlock]); + for (int j = 0; j < miniBlockSizeInValues; j += 8) { + ByteBuffer buffer = inputStream.slice(packer.getBitWidth()); + if (buffer.hasArray()) { + packer.unpack8Values( + buffer.array(), buffer.arrayOffset() + buffer.position(), unpackedValuesBuffer, j); + } else { + packer.unpack8Values(buffer, buffer.position(), unpackedValuesBuffer, j); + } + } + remainingInMiniBlock = miniBlockSizeInValues; + currentMiniBlock++; + } + + // From org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader + private void readBitWidthsForMiniBlocks() { + for (int i = 0; i < miniBlocksPerBlock; i++) { + try { + bitWidths[i] = BytesUtils.readIntLittleEndianOnOneByte(inputStream); + } catch (IOException e) { + throw new ParquetDecodingException("Can not decode bitwidth in block header", e); + } + } + } + + /** A functional interface to write long values to into a FieldVector */ + @FunctionalInterface + interface IntegerOutputWriter { + + /** + * A functional interface that can be used to write a long value to a specified row in a + * FieldVector + * + * @param vec a FieldVector to write the value into + * @param index The offset to write to + * @param val value to write + */ + void write(FieldVector vec, long index, long val); + } +} diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java index 99cf2dc45864..be1a3324ae43 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java @@ -93,16 +93,21 @@ protected void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, i throw new ParquetDecodingException("could not read page in col " + desc, e); } } else { - if (dataEncoding == Encoding.PLAIN) { - valuesReader = new VectorizedPlainValuesReader(); - } else { - throw new UnsupportedOperationException( - "Cannot support vectorized reads for column " - + desc - + " with " - + "encoding " - + dataEncoding - + ". Disable vectorized reads to read this table/file"); + switch (dataEncoding) { + case PLAIN: + valuesReader = new VectorizedPlainValuesReader(); + break; + case DELTA_BINARY_PACKED: + valuesReader = new VectorizedDeltaEncodedValuesReader(); + break; + default: + throw new UnsupportedOperationException( + "Cannot support vectorized reads for column " + + desc + + " with " + + "encoding " + + dataEncoding + + ". Disable vectorized reads to read this table/file"); } try { valuesReader.initFromPage(valueCount, in); diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java index 0f85101a5b79..c7dbe8de7b92 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetDefinitionLevelReader.java @@ -223,7 +223,7 @@ protected void nextRleBatch( int idx, int numValues, byte[] byteArray) { - setNextNValuesInVector(typeWidth, nullabilityHolder, valuesReader, idx, vector, numValues); + setNextNValuesInVector(nullabilityHolder, valuesReader, idx, vector, numValues); } @Override @@ -252,6 +252,29 @@ protected void nextPackedBatch( protected abstract void nextVal( FieldVector vector, int idx, VectorizedValuesReader valuesReader, Mode mode); + + public abstract void nextVals( + FieldVector vector, int rowId, VectorizedValuesReader valuesReader, int total); + + private void setNextNValuesInVector( + NullabilityHolder nullabilityHolder, + VectorizedValuesReader valuesReader, + int bufferIdx, + FieldVector vector, + int numValues) { + ArrowBuf validityBuffer = vector.getValidityBuffer(); + if (currentValue == maxDefLevel) { + nextVals(vector, bufferIdx, valuesReader, numValues); + nullabilityHolder.setNotNulls(bufferIdx, numValues); + if (setArrowValidityVector) { + for (int i = 0; i < numValues; i++) { + BitVectorHelper.setBit(validityBuffer, bufferIdx + i); + } + } + } else { + setNulls(nullabilityHolder, bufferIdx, numValues, validityBuffer); + } + } } class LongReader extends NumericBaseReader { @@ -279,6 +302,12 @@ protected void nextDictEncodedVal( .setLong((long) idx * typeWidth, dict.decodeToLong(reader.readInteger())); } } + + @Override + public void nextVals( + FieldVector vector, int rowId, VectorizedValuesReader valuesReader, int total) { + valuesReader.readLongs(total, vector, rowId); + } } class DoubleReader extends NumericBaseReader { @@ -306,6 +335,12 @@ protected void nextDictEncodedVal( .setDouble((long) idx * typeWidth, dict.decodeToDouble(reader.readInteger())); } } + + @Override + public void nextVals( + FieldVector vector, int rowId, VectorizedValuesReader valuesReader, int total) { + valuesReader.readDoubles(total, vector, rowId); + } } class FloatReader extends NumericBaseReader { @@ -333,6 +368,12 @@ protected void nextDictEncodedVal( .setFloat((long) idx * typeWidth, dict.decodeToFloat(reader.readInteger())); } } + + @Override + public void nextVals( + FieldVector vector, int rowId, VectorizedValuesReader valuesReader, int total) { + valuesReader.readFloats(total, vector, rowId); + } } class IntegerReader extends NumericBaseReader { @@ -362,6 +403,12 @@ protected void nextDictEncodedVal( .setInt((long) idx * typeWidth, dict.decodeToInt(reader.readInteger())); } } + + @Override + public void nextVals( + FieldVector vector, int rowId, VectorizedValuesReader valuesReader, int total) { + valuesReader.readIntegers(total, vector, rowId); + } } abstract class BaseReader extends CommonReader { @@ -647,28 +694,6 @@ private void setNulls( } } - private void setNextNValuesInVector( - int typeWidth, - NullabilityHolder nullabilityHolder, - VectorizedValuesReader valuesReader, - int bufferIdx, - FieldVector vector, - int numValues) { - ArrowBuf validityBuffer = vector.getValidityBuffer(); - if (currentValue == maxDefLevel) { - ByteBuffer buffer = valuesReader.readBinary(numValues * typeWidth).toByteBuffer(); - vector.getDataBuffer().setBytes((long) bufferIdx * typeWidth, buffer); - nullabilityHolder.setNotNulls(bufferIdx, numValues); - if (setArrowValidityVector) { - for (int i = 0; i < numValues; i++) { - BitVectorHelper.setBit(validityBuffer, bufferIdx + i); - } - } - } else { - setNulls(nullabilityHolder, bufferIdx, numValues, validityBuffer); - } - } - LongReader longReader() { return new LongReader(); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPlainValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPlainValuesReader.java index ac98da17f2ab..764b2fc353e3 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPlainValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPlainValuesReader.java @@ -25,11 +25,6 @@ class VectorizedPlainValuesReader extends ValuesAsBytesReader implements VectorizedValuesReader { - public static final int INT_SIZE = 4; - public static final int LONG_SIZE = 8; - public static final int FLOAT_SIZE = 4; - public static final int DOUBLE_SIZE = 8; - VectorizedPlainValuesReader() {} @Override diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedValuesReader.java index f8b7c92f5999..7c23149b18ab 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedValuesReader.java @@ -31,6 +31,11 @@ */ interface VectorizedValuesReader { + int INT_SIZE = 4; + int LONG_SIZE = 8; + int FLOAT_SIZE = 4; + int DOUBLE_SIZE = 8; + /** Read a single boolean */ boolean readBoolean(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java index ff9d624ae68f..d7f7f9c68d21 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java @@ -295,12 +295,14 @@ public void testReadsForTypePromotedColumns() throws Exception { public void testSupportedReadsForParquetV2() throws Exception { // Float and double column types are written using plain encoding with Parquet V2, // also Parquet V2 will dictionary encode decimals that use fixed length binary - // (i.e. decimals > 8 bytes) + // (i.e. decimals > 8 bytes). Int and long types use DELTA_BINARY_PACKED. Schema schema = new Schema( optional(102, "float_data", Types.FloatType.get()), optional(103, "double_data", Types.DoubleType.get()), - optional(104, "decimal_data", Types.DecimalType.of(25, 5))); + optional(104, "decimal_data", Types.DecimalType.of(25, 5)), + optional(105, "int_data", Types.IntegerType.get()), + optional(106, "long_data", Types.LongType.get())); File dataFile = File.createTempFile("junit", null, temp.toFile()); assertThat(dataFile.delete()).as("Delete should succeed").isTrue(); @@ -314,8 +316,7 @@ public void testSupportedReadsForParquetV2() throws Exception { @Test public void testUnsupportedReadsForParquetV2() throws Exception { - // Longs, ints, string types etc use delta encoding and which are not supported for vectorized - // reads + // Some types use delta encoding and which are not supported for vectorized reads Schema schema = new Schema(SUPPORTED_PRIMITIVES.fields()); File dataFile = File.createTempFile("junit", null, temp.toFile()); assertThat(dataFile.delete()).as("Delete should succeed").isTrue();