You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/01/02 20:38:24 UTC

[GitHub] [incubator-iceberg] samarthjain opened a new pull request #723: Arrow changes for supporting vectorized reads

samarthjain opened a new pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723
 
 
   Co-authored-by: gautamkowshik@gmail.com
   Co-authored-by: anjalinorwood@gmail.com

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367734806
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
 
 Review comment:
   This class is really huge and there are two main modes it is used for: definition levels and dictionary indexes. Can we split this into two separate classes for those two uses? I don't think that would hurt performance, but it would be a bit easier to read. We could move the common things into a base class, too.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368132185
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
 
 Review comment:
   It doesn't show up in github or stash, though. Without named parameters, I think it's still a best practice to add comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367720714
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
 
 Review comment:
   Style: Can we update these names to match the others? Also, I don't think we need to have each argument on a separate line.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380816971
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] decimalBytes = dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe();
+              byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+              System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP) This
 
 Review comment:
   @samarthjain, did you see this question?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380402171
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,247 @@
+/*
+ * 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 org.apache.parquet.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.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public class BaseVectorizedParquetValuesReader extends ValuesReader {
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  enum MODE {
 
 Review comment:
   What about using `Mode` instead of `MODE`? Usually symbols are all caps, while types are camel case.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367718286
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,565 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class VectorizedPageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(VectorizedPageIterator.class);
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
 
 Review comment:
   We should also try to make a superclass for this iterator and the row-based `PageIterator` because there is a fair amount of duplication here as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367730465
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
 
 Review comment:
   This shouldn't allocate a new byte array each time through the loop. It should create the buffer at the start of the method call and reuse it for each value. We could probably also cache these to avoid the allocation entirely, but that would require a size-based buffer cache and a thread-local.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367721667
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
 
 Review comment:
   Can we prefix with `this.` to show that this is setting an instance field?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380404000
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java
 ##########
 @@ -0,0 +1,408 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.Dictionary;
+
+public class VectorizedDictionaryEncodedParquetValuesReader extends BaseVectorizedParquetValuesReader {
+
+  public VectorizedDictionaryEncodedParquetValuesReader(int maxDefLevel, boolean setValidityVector) {
+    super(maxDefLevel, setValidityVector);
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  void readBatchOfDictionaryIds(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead,
 
 Review comment:
   Also, `numValsInVector` is really an offset. Can we use a better name, like `startOffset`? Looks like `index` is used elsewhere. That's a good name, too.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380399902
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private ReadType readType;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize,
+      boolean setArrowValidityVector) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize, setArrowValidityVector);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.vectorizedColumnIterator = null;
+  }
+
+  private enum ReadType {
+    FIXED_LENGTH_DECIMAL, INT_LONG_BACKED_DECIMAL, VARCHAR, VARBINARY, FIXED_WIDTH_BINARY,
+    BOOLEAN, INT, LONG, FLOAT, DOUBLE
+  }
+
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      allocateFieldVector();
+      nullabilityHolder = new NullabilityHolder(batchSize);
+    } else {
+      vec.setValueCount(0);
+      nullabilityHolder.reset();
+    }
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        switch (readType) {
+          case FIXED_LENGTH_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case INT_LONG_BACKED_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case VARBINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case VARCHAR:
+            ((IcebergArrowVectors.VarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case FIXED_WIDTH_BINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+            break;
+          case BOOLEAN:
+            vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+            break;
+          case INT:
+            vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+            break;
+          case LONG:
+            vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+            break;
+          case FLOAT:
+            vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+            break;
+          case DOUBLE:
+            vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+            break;
+        }
+      }
+    }
+    Preconditions.checkState(vec.getValueCount() == numValsToRead,
+        "Number of values read, %s, does not equal expected, %s", vec.getValueCount(), numValsToRead);
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private void allocateFieldVector() {
+    if (allPagesDictEncoded) {
+      Field field = new Field(
+          icebergField.name(),
+          new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null),
+          null);
+      this.vec = field.createVector(rootAlloc);
+      ((IntVector) vec).allocateNew(batchSize);
+      typeWidth = IntVector.TYPE_WIDTH;
+    } else {
+      PrimitiveType primitive = columnDescriptor.getPrimitiveType();
+      if (primitive.getOriginalType() != null) {
+        switch (columnDescriptor.getPrimitiveType().getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+          case BSON:
+            this.vec = new IcebergArrowVectors.VarcharArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
+            vec.allocateNewSafe();
+            readType = ReadType.VARCHAR;
+            typeWidth = UNKNOWN_WIDTH;
 
 Review comment:
   Minor: it helps readability to set instance fields using `this.typeWidth = ...`. That way we can see it's a side-effect of the method call and not just a local variable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367715864
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java
 ##########
 @@ -0,0 +1,258 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Vectorized version of the ColumnIterator that reads column values in data pages of a column in a row group in a
+ * batched fashion.
+ */
+public class VectorizedColumnIterator {
+
+  private final ColumnDescriptor desc;
+  private final VectorizedPageIterator vectorizedPageIterator;
+
+  // state reset for each row group
+  private PageReader columnPageReader = null;
+  private long totalValuesCount = 0L;
+  private long valuesRead = 0L;
+  private long advanceNextPageCount = 0L;
+  private final int batchSize;
+
+  public VectorizedColumnIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
+    this.batchSize = batchSize;
+    this.vectorizedPageIterator = new VectorizedPageIterator(desc, writerVersion, batchSize);
+  }
+
+  public Dictionary setRowGroupInfo(PageReadStore store, boolean allPagesDictEncoded) {
 
 Review comment:
   Why does this need to be a `PageReadStore` instead of a `PageReader`? This implementation is nearly identical to `setPageSource` in the non-vectorized read path. I think we could refactor into a superclass and reuse `setPageSource` and `readDictionary` if we were to refactor into a common superclass. Probably don't need to do that in this PR, but it would be nice to avoid unnecessary drift between the two read implementations.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387370237
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java
 ##########
 @@ -0,0 +1,202 @@
+/*
+ * 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.parquet;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class BasePageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(BasePageIterator.class);
+
+  protected final ColumnDescriptor desc;
+  protected final String writerVersion;
+
+  // iterator state
+  protected boolean hasNext = false;
+  protected int triplesRead = 0;
+  protected int currentDL = 0;
+  protected int currentRL = 0;
+
+  // page bookkeeping
+  protected Dictionary dictionary = null;
+  protected DataPage page = null;
+  protected int triplesCount = 0;
+  protected Encoding valueEncoding = null;
+  protected IntIterator definitionLevels = null;
+  protected IntIterator repetitionLevels = null;
+  protected ValuesReader vectorizedDefinitionLevelReader = null;
+  protected ValuesReader values = null;
+
+  protected BasePageIterator(ColumnDescriptor descriptor, String writerVersion) {
+    this.desc = descriptor;
+    this.writerVersion = writerVersion;
+  }
+
+  protected abstract void reset();
+
+  protected abstract boolean supportsVectorizedReads();
+
+  protected abstract IntIterator newNonVectorizedDefinitionLevelReader(ValuesReader dlReader);
+
+  protected abstract ValuesReader newVectorizedDefinitionLevelReader(ColumnDescriptor descriptor);
+
+  protected abstract void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount);
+
+  public void setPage(DataPage page) {
+    Preconditions.checkNotNull(page, "Cannot read from null page");
+    this.page = page;
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    this.hasNext = triplesRead < triplesCount;
+  }
+
+  protected void initFromPage(DataPageV1 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    ValuesReader dlReader = null;
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      dlReader = initPage.getDlEncoding().getValuesReader(desc, ValuesType.DEFINITION_LEVEL);
+      this.definitionLevels = newNonVectorizedDefinitionLevelReader(dlReader);
+    }
+    ValuesReader rlReader = initPage.getRlEncoding().getValuesReader(desc, ValuesType.REPETITION_LEVEL);
+    this.repetitionLevels = new PageIterator.ValuesReaderIntIterator(rlReader);
+    try {
+      BytesInput bytes = initPage.getBytes();
+      LOG.debug("page size {} bytes and {} records", bytes.size(), triplesCount);
+      LOG.debug("reading repetition levels at 0");
+      ByteBufferInputStream in = bytes.toInputStream();
+      rlReader.initFromPage(triplesCount, in);
+      LOG.debug("reading definition levels at {}", in.position());
+      if (supportsVectorizedReads()) {
+        this.vectorizedDefinitionLevelReader.initFromPage(triplesCount, in);
+      } else {
+        dlReader.initFromPage(triplesCount, in);
+      }
+      LOG.debug("reading data at {}", in.position());
+      initDataReader(initPage.getValueEncoding(), in, initPage.getValueCount());
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  protected void initFromPage(DataPageV2 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    this.repetitionLevels = newRLEIterator(desc.getMaxRepetitionLevel(), initPage.getRepetitionLevels());
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      this.definitionLevels = newRLEIterator(desc.getMaxDefinitionLevel(), initPage.getDefinitionLevels());
+    }
+    LOG.debug("page data size {} bytes and {} records", initPage.getData().size(), triplesCount);
+    try {
+      initDataReader(initPage.getDataEncoding(), initPage.getData().toInputStream(), triplesCount);
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
+    try {
+      if (maxLevel == 0) {
+        return new NullIntIterator();
+      }
+      return new RLEIntIterator(
+          new RunLengthBitPackingHybridDecoder(
+              BytesUtils.getWidthFromMaxInt(maxLevel),
+              bytes.toInputStream()));
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read levels in page for col " + desc, e);
+    }
+  }
+
+  public void setDictionary(Dictionary dict) {
+    this.dictionary = dict;
+  }
+
+  protected abstract static class IntIterator {
+    abstract int nextInt();
+  }
+
+  static class ValuesReaderIntIterator extends IntIterator {
 
 Review comment:
   @samarthjain, let's move these if they aren't used by the vectorized path back into the row-based path.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367655060
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorHolder.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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;
+
+import javax.annotation.Nullable;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+
+/**
+ * Container class for holding the Arrow vector holding a batch of values along with other state needed for reading
+ * values out of it.
+ */
+public class VectorHolder {
 
 Review comment:
   Why handle nullability and dictionary encoding differently here than with the Iceberg vector classes above? This wraps vectors and those extend vectors.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367667068
 
 

 ##########
 File path: build.gradle
 ##########
 @@ -228,6 +228,21 @@ project(':iceberg-parquet') {
   }
 }
 
+project(':iceberg-arrow') {
+  dependencies {
+    compile project(':iceberg-api')
+    compile project(':iceberg-parquet')
+
+    compile("org.apache.arrow:arrow-vector") {
+      exclude group: 'io.netty', module: 'netty-buffer'
+      exclude group: 'io.netty', module: 'netty-common'
 
 Review comment:
   Why exclude Netty? Isn't this needed for tests?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] baibaichen commented on pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
baibaichen commented on pull request #723:
URL: https://github.com/apache/iceberg/pull/723#issuecomment-689673810


   @samarthjain 
   
   Is there any design documentation about reading parquet into Arrow format?  I found iceberg rewrite parquet vector reader, what is the reason?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380402271
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,247 @@
+/*
+ * 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 org.apache.parquet.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.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public class BaseVectorizedParquetValuesReader extends ValuesReader {
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  MODE mode;
 
 Review comment:
   Are these accessed outside of subclasses? If not, then we should use `protected` instead of package-private.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367664512
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.isFixedLengthDecimal = false;
+    this.isVarWidthType = false;
+    this.isFixedWidthBinary = false;
+    this.isBooleanType = false;
+    this.isPaddedDecimal = false;
+    this.isIntType = false;
+    this.isLongType = false;
+    this.isFloatType = false;
+    this.isDoubleType = false;
+    this.vectorizedColumnIterator = null;
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      typeWidth = allocateFieldVector();
 
 Review comment:
   Right now, allocation and reads use separate logic. Reads use the `isXType` booleans, while `allocateFieldVector` uses the Parquet type.
   
   I think I understand why those are slightly different: allocation needs to create, for example, a `DateDayVector` but reading calls `nextBatchIntergers` because it writes into the underlying buffer directly.
   
   I think that having these completely decoupled could lead to problems, though. I'd rather allocate a vector and then base the type of read on the vector that was produced. Something like this:
   
   ```
     public VectorizedArrowReader(...) {
       ...
       this.vector = allocateFieldVector();
       if (this.vector instanceof IntVector || this.vector instanceof DateDayVector) {
         this.readMethod = ReadMethod.INT;
       } else if (this.vector instanceof LongVector || this.vector instanceof TimeStampMicroTZVector) {
         this.readMethod = ReadMethod.LONG;
       }
     }
   ```
   
   The logic would probably be more complicated because it needs to rely on both the incoming Parquet encoding and the outgoing vector type. But the main thing is that it would be good to check that the vector and the read method are compatible.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367671357
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java
 ##########
 @@ -0,0 +1,258 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Vectorized version of the ColumnIterator that reads column values in data pages of a column in a row group in a
+ * batched fashion.
+ */
+public class VectorizedColumnIterator {
+
+  private final ColumnDescriptor desc;
+  private final VectorizedPageIterator vectorizedPageIterator;
+
+  // state reset for each row group
+  private PageReader columnPageReader = null;
+  private long totalValuesCount = 0L;
+  private long valuesRead = 0L;
+  private long advanceNextPageCount = 0L;
+  private final int batchSize;
+
+  public VectorizedColumnIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
+    this.batchSize = batchSize;
+    this.vectorizedPageIterator = new VectorizedPageIterator(desc, writerVersion, batchSize);
+  }
+
+  public Dictionary setRowGroupInfo(PageReadStore store, boolean allPagesDictEncoded) {
+    this.columnPageReader = store.getPageReader(desc);
+    this.totalValuesCount = columnPageReader.getTotalValueCount();
 
 Review comment:
   Nevermind, I see that the other is passed the PageReader instead of getting it from the PageReadStore like this one does.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380405331
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java
 ##########
 @@ -0,0 +1,408 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.Dictionary;
+
+public class VectorizedDictionaryEncodedParquetValuesReader extends BaseVectorizedParquetValuesReader {
+
+  public VectorizedDictionaryEncodedParquetValuesReader(int maxDefLevel, boolean setValidityVector) {
+    super(maxDefLevel, setValidityVector);
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
 
 Review comment:
   None of the methods in this class should be checking definition levels, right? Those are checked by the definition level reader that is used to call these methods. If that's right, then could you add that to class-level Javadoc? I think explaining how this class is used would be really useful for future maintenance.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367718075
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,565 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class VectorizedPageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(VectorizedPageIterator.class);
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
+    this.writerVersion = writerVersion;
+  }
+
+  private final ColumnDescriptor desc;
+  private final String writerVersion;
+
+  // iterator state
+  private boolean hasNext = false;
+  private int triplesRead = 0;
+
+  // page bookkeeping
+  private Dictionary dictionary = null;
+  private DataPage page = null;
+  private int triplesCount = 0;
+
+  // Needed once we add support for complex types. Unused for now.
+  private IntIterator repetitionLevels = null;
+  private int currentRL = 0;
+
+  private VectorizedParquetValuesReader definitionLevelReader;
+  private boolean eagerDecodeDictionary;
+  private ValuesAsBytesReader plainValuesReader = null;
+  private VectorizedParquetValuesReader dictionaryEncodedValuesReader = null;
+  private boolean allPagesDictEncoded;
+
+  public void setPage(DataPage dataPage) {
+    this.page = Preconditions.checkNotNull(dataPage, "Cannot read from null page");
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    advance();
+  }
+
+  // Dictionary is set per row group
+  public void setDictionaryForColumn(Dictionary dict, boolean allDictEncoded) {
+    this.dictionary = dict;
+    this.allPagesDictEncoded = allDictEncoded;
+  }
+
+  public void reset() {
+    this.page = null;
+    this.triplesCount = 0;
+    this.triplesRead = 0;
+    this.repetitionLevels = null;
+    this.plainValuesReader = null;
+    this.definitionLevelReader = null;
+    this.hasNext = false;
+  }
+
+  public int currentPageCount() {
+    return triplesCount;
+  }
+
+  public boolean hasNext() {
+    return hasNext;
+  }
+
+  /**
+   * Method for reading a batch of dictionary ids from the dicitonary encoded data pages. Like definition levels,
+   * dictionary ids in Parquet are RLE/bin-packed encoded as well.
+   */
+  public int nextBatchDictionaryIds(
+      final IntVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    definitionLevelReader.readBatchOfDictionaryIds(
+        vector,
+        numValsInVector,
+        actualBatchSize,
+        holder,
+        dictionaryEncodedValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT32 data type
+   */
+  public int nextBatchIntegers(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT64 data type
+   */
+  public int nextBatchLongs(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of FLOAT data type.
+   */
+  public int nextBatchFloats(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of DOUBLE data type
+   */
+  public int nextBatchDoubles(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  private int getActualBatchSize(int expectedBatchSize) {
+    return Math.min(expectedBatchSize, triplesCount - triplesRead);
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by INT32 and INT64 parquet data types. Since Arrow stores all
+   * decimals in 16 bytes, byte arrays are appropriately padded before being written to Arrow data buffers.
+   */
+  public int nextBatchIntLongBackedDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedIntLongBackedDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfIntLongBackedDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by fixed length byte array parquet data type. Arrow stores all
+   * decimals in 16 bytes. This method provides the necessary padding to the decimals read. Moreover, Arrow interprets
+   * the decimals in Arrow buffer as little endian. Parquet stores fixed length decimals as big endian. So, this method
+   * uses {@link DecimalVector#setBigEndian(int, byte[])} method so that the data in Arrow vector is indeed little
+   * endian.
+   */
+  public int nextBatchFixedLengthDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of variable width data type (ENUM, JSON, UTF8, BSON).
+   */
+  public int nextBatchVarWidthType(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of fixed width binary type (e.g. BYTE[7]). Spark does not support fixed width binary
+   * data type. To work around this limitation, the data is read as fixed width binary from parquet and stored in a
+   * {@link VarBinaryVector} in Arrow.
+   */
+  public int nextBatchFixedWidthBinary(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of booleans.
+   */
+  public int nextBatchBoolean(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    definitionLevelReader.readBatchOfBooleans(vector, numValsInVector, actualBatchSize,
+        nullabilityHolder, plainValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  private void advance() {
 
 Review comment:
   Looks like you don't need this method any more. It is only used in `setPage` and the `nextBatch` methods all use `this.hasNext = triplesRead < triplesCount;`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380820858
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java
 ##########
 @@ -255,4 +260,19 @@ public static boolean hasNonDictionaryPages(ColumnChunkMetaData meta) {
       return true;
     }
   }
+
+  public static Dictionary readDictionary(ColumnDescriptor desc, PageReader pageSource) {
+    DictionaryPage dictionaryPage = pageSource.readDictionaryPage();
+    if (dictionaryPage != null) {
+      try {
+        return dictionaryPage.getEncoding().initDictionary(desc, dictionaryPage);
+//        if (converter.hasDictionarySupport()) {
+//          converter.setDictionary(dictionary);
+//        }
 
 Review comment:
   I think we can remove these comment lines.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380824039
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/VectorizedReader.java
 ##########
 @@ -44,8 +44,13 @@
   void setRowGroupInfo(PageReadStore pages, Map<ColumnPath, ColumnChunkMetaData> metadata);
 
   /**
-   * Set up the reader to reuse the underlying containers used for storing batches
+   * Setup the reader to reuse the underlying containers used for storing batches
 
 Review comment:
   This was correct before; "setup" is a noun and "set up" is the verb form.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387231267
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java
 ##########
 @@ -23,26 +23,18 @@
 import java.io.IOException;
 import org.apache.parquet.CorruptDeltaByteArrays;
 import org.apache.parquet.bytes.ByteBufferInputStream;
-import org.apache.parquet.bytes.BytesInput;
-import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.Dictionary;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.ValuesType;
 import org.apache.parquet.column.page.DataPage;
 import org.apache.parquet.column.page.DataPageV1;
 import org.apache.parquet.column.page.DataPageV2;
 import org.apache.parquet.column.values.RequiresPreviousReader;
 import org.apache.parquet.column.values.ValuesReader;
-import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
 import org.apache.parquet.io.ParquetDecodingException;
 import org.apache.parquet.io.api.Binary;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class PageIterator<T> implements TripleIterator<T> {
-  private static final Logger LOG = LoggerFactory.getLogger(PageIterator.class);
 
+abstract class PageIterator<T> extends BasePageIterator implements TripleIterator<T> {
 
 Review comment:
   PageIterator was abstract before as well. 
   https://github.com/apache/incubator-iceberg/blob/master/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java#L43
   
   The constructor is private too, so this class was meant to be instantiated only using the `newIterator` method. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380400987
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private ReadType readType;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize,
+      boolean setArrowValidityVector) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize, setArrowValidityVector);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.vectorizedColumnIterator = null;
+  }
+
+  private enum ReadType {
+    FIXED_LENGTH_DECIMAL, INT_LONG_BACKED_DECIMAL, VARCHAR, VARBINARY, FIXED_WIDTH_BINARY,
+    BOOLEAN, INT, LONG, FLOAT, DOUBLE
+  }
+
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      allocateFieldVector();
+      nullabilityHolder = new NullabilityHolder(batchSize);
+    } else {
+      vec.setValueCount(0);
+      nullabilityHolder.reset();
+    }
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        switch (readType) {
+          case FIXED_LENGTH_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case INT_LONG_BACKED_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case VARBINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case VARCHAR:
+            ((IcebergArrowVectors.VarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case FIXED_WIDTH_BINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+            break;
+          case BOOLEAN:
+            vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+            break;
+          case INT:
+            vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+            break;
+          case LONG:
+            vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+            break;
+          case FLOAT:
+            vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+            break;
+          case DOUBLE:
+            vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+            break;
+        }
+      }
+    }
+    Preconditions.checkState(vec.getValueCount() == numValsToRead,
+        "Number of values read, %s, does not equal expected, %s", vec.getValueCount(), numValsToRead);
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private void allocateFieldVector() {
+    if (allPagesDictEncoded) {
+      Field field = new Field(
+          icebergField.name(),
+          new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null),
+          null);
+      this.vec = field.createVector(rootAlloc);
+      ((IntVector) vec).allocateNew(batchSize);
+      typeWidth = IntVector.TYPE_WIDTH;
+    } else {
+      PrimitiveType primitive = columnDescriptor.getPrimitiveType();
+      if (primitive.getOriginalType() != null) {
+        switch (columnDescriptor.getPrimitiveType().getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+          case BSON:
+            this.vec = new IcebergArrowVectors.VarcharArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
+            vec.allocateNewSafe();
+            readType = ReadType.VARCHAR;
+            typeWidth = UNKNOWN_WIDTH;
+            break;
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((IntVector) vec).allocateNew(batchSize);
+            readType = ReadType.INT;
+            typeWidth = IntVector.TYPE_WIDTH;
+            break;
+          case DATE:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((DateDayVector) vec).allocateNew(batchSize);
+            readType = ReadType.INT;
+            typeWidth = IntVector.TYPE_WIDTH;
+            break;
+          case INT_64:
+          case TIMESTAMP_MILLIS:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((BigIntVector) vec).allocateNew(batchSize);
+            readType = ReadType.LONG;
+            typeWidth = BigIntVector.TYPE_WIDTH;
+            break;
+          case TIMESTAMP_MICROS:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((TimeStampMicroTZVector) vec).allocateNew(batchSize);
+            readType = ReadType.LONG;
+            typeWidth = BigIntVector.TYPE_WIDTH;
+            break;
+          case DECIMAL:
+            DecimalMetadata decimal = primitive.getDecimalMetadata();
+            this.vec = new IcebergArrowVectors.DecimalArrowVector(icebergField.name(), rootAlloc,
+                decimal.getPrecision(), decimal.getScale());
+            ((DecimalVector) vec).allocateNew(batchSize);
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                readType = ReadType.FIXED_LENGTH_DECIMAL;
+                typeWidth = primitive.getTypeLength();
+                break;
+              case INT64:
+                readType = ReadType.INT_LONG_BACKED_DECIMAL;
+                typeWidth = BigIntVector.TYPE_WIDTH;
+                break;
+              case INT32:
+                readType = ReadType.INT_LONG_BACKED_DECIMAL;
+                typeWidth = IntVector.TYPE_WIDTH;
+                break;
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+            break;
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      } else {
+        switch (primitive.getPrimitiveTypeName()) {
+          case FIXED_LEN_BYTE_ARRAY:
+            int len = ((Types.FixedType) icebergField.type()).length();
+            this.vec = new IcebergArrowVectors.VarBinaryArrowVector(icebergField.name(), rootAlloc);
+            int factor = (len + DEFAULT_RECORD_BYTE_COUNT - 1) / DEFAULT_RECORD_BYTE_COUNT;
+            vec.setInitialCapacity(batchSize * factor);
+            vec.allocateNew();
+            readType = ReadType.FIXED_WIDTH_BINARY;
+            typeWidth = len;
+            break;
+          case BINARY:
+            this.vec = new IcebergArrowVectors.VarBinaryArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
 
 Review comment:
   Similar to the string case, I think this default size should be a constant somewhere.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380400024
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private ReadType readType;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize,
+      boolean setArrowValidityVector) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize, setArrowValidityVector);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.vectorizedColumnIterator = null;
+  }
+
+  private enum ReadType {
+    FIXED_LENGTH_DECIMAL, INT_LONG_BACKED_DECIMAL, VARCHAR, VARBINARY, FIXED_WIDTH_BINARY,
+    BOOLEAN, INT, LONG, FLOAT, DOUBLE
+  }
+
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      allocateFieldVector();
+      nullabilityHolder = new NullabilityHolder(batchSize);
+    } else {
+      vec.setValueCount(0);
+      nullabilityHolder.reset();
+    }
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        switch (readType) {
+          case FIXED_LENGTH_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case INT_LONG_BACKED_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case VARBINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case VARCHAR:
+            ((IcebergArrowVectors.VarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case FIXED_WIDTH_BINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+            break;
+          case BOOLEAN:
+            vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+            break;
+          case INT:
+            vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+            break;
+          case LONG:
+            vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+            break;
+          case FLOAT:
+            vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+            break;
+          case DOUBLE:
+            vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+            break;
+        }
+      }
+    }
+    Preconditions.checkState(vec.getValueCount() == numValsToRead,
+        "Number of values read, %s, does not equal expected, %s", vec.getValueCount(), numValsToRead);
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private void allocateFieldVector() {
+    if (allPagesDictEncoded) {
+      Field field = new Field(
+          icebergField.name(),
+          new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null),
+          null);
+      this.vec = field.createVector(rootAlloc);
+      ((IntVector) vec).allocateNew(batchSize);
+      typeWidth = IntVector.TYPE_WIDTH;
+    } else {
+      PrimitiveType primitive = columnDescriptor.getPrimitiveType();
+      if (primitive.getOriginalType() != null) {
+        switch (columnDescriptor.getPrimitiveType().getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+          case BSON:
+            this.vec = new IcebergArrowVectors.VarcharArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
+            vec.allocateNewSafe();
+            readType = ReadType.VARCHAR;
+            typeWidth = UNKNOWN_WIDTH;
+            break;
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
 
 Review comment:
   Minor: should this reader convert the field just once, or does this need to be done each time a new vector is allocated?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380815313
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,444 @@
+/*
+ * 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 org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.BasePageIterator;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+public class VectorizedPageIterator extends BasePageIterator {
+  private final boolean setArrowValidityVector;
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, boolean setValidityVector) {
+    super(desc, writerVersion);
+    this.setArrowValidityVector = setValidityVector;
+  }
+
+  private boolean eagerDecodeDictionary;
+  private ValuesAsBytesReader plainValuesReader = null;
+  private VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader = null;
+  private boolean allPagesDictEncoded;
+
+  // Dictionary is set per row group
+  public void setDictionaryForColumn(Dictionary dict, boolean allDictEncoded) {
+    this.dictionary = dict;
+    this.allPagesDictEncoded = allDictEncoded;
+  }
+
+  @Override
+  protected void reset() {
+    this.page = null;
+    this.triplesCount = 0;
+    this.triplesRead = 0;
+    this.repetitionLevels = null;
+    this.plainValuesReader = null;
+    this.vectorizedDefinitionLevelReader = null;
+    this.hasNext = false;
+  }
+
+  public int currentPageCount() {
+    return triplesCount;
+  }
+
+  public boolean hasNext() {
+    return hasNext;
+  }
+
+  /**
+   * Method for reading a batch of dictionary ids from the dicitonary encoded data pages. Like definition levels,
+   * dictionary ids in Parquet are RLE/bin-packed encoded as well.
+   */
+  public int nextBatchDictionaryIds(
+      final IntVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    ((VectorizedParquetValuesReader) vectorizedDefinitionLevelReader).readBatchOfDictionaryIds(
 
 Review comment:
   It seems strange to me that this is always cast to a `VectorizedParquetValuesReader` in this class. Why not make the field that type?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367730972
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] decimalBytes = dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe();
+              byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+              System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe();
 
 Review comment:
   Same here. It would be great to follow up and get rid of this copy.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367644532
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/IcebergDecimalArrowVector.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.iceberg.arrow.vectorized;
+
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.DecimalVector;
+
+/**
+ * Extension of Arrow's @{@link DecimalVector}. The whole reason of having this implementation is to override the
+ * expensive {@link DecimalVector#isSet(int)} method used by  {@link DecimalVector#getObject(int)}.
+ */
+public class IcebergDecimalArrowVector extends DecimalVector {
 
 Review comment:
   Looks like there are quite a few of these classes that exist just to override Arrow's validity vector check. Can we locate these classes somewhere to signal that they all have the same purpose? That could be making them static classes in a container class or moving them into a different sub-package.
   
   I like the container class approach:
   
   ```java
   /**
    * Iceberg extensions for Arrow vectors that override the {@code isSet(int)} methods for performance.
    */
   public class IcebergArrowVectors {
     public static class DecimalArrowVector extends DecimalVector { ... }
   
     public static class VarBinaryArrowVector extends VarBinaryVector { ... }
   }
   ```
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380399791
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private ReadType readType;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize,
+      boolean setArrowValidityVector) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize, setArrowValidityVector);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.vectorizedColumnIterator = null;
+  }
+
+  private enum ReadType {
+    FIXED_LENGTH_DECIMAL, INT_LONG_BACKED_DECIMAL, VARCHAR, VARBINARY, FIXED_WIDTH_BINARY,
+    BOOLEAN, INT, LONG, FLOAT, DOUBLE
+  }
+
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      allocateFieldVector();
+      nullabilityHolder = new NullabilityHolder(batchSize);
+    } else {
+      vec.setValueCount(0);
+      nullabilityHolder.reset();
+    }
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        switch (readType) {
+          case FIXED_LENGTH_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case INT_LONG_BACKED_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case VARBINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case VARCHAR:
+            ((IcebergArrowVectors.VarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case FIXED_WIDTH_BINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+            break;
+          case BOOLEAN:
+            vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+            break;
+          case INT:
+            vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+            break;
+          case LONG:
+            vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+            break;
+          case FLOAT:
+            vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+            break;
+          case DOUBLE:
+            vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+            break;
+        }
+      }
+    }
+    Preconditions.checkState(vec.getValueCount() == numValsToRead,
+        "Number of values read, %s, does not equal expected, %s", vec.getValueCount(), numValsToRead);
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private void allocateFieldVector() {
+    if (allPagesDictEncoded) {
+      Field field = new Field(
+          icebergField.name(),
+          new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null),
+          null);
+      this.vec = field.createVector(rootAlloc);
+      ((IntVector) vec).allocateNew(batchSize);
+      typeWidth = IntVector.TYPE_WIDTH;
+    } else {
+      PrimitiveType primitive = columnDescriptor.getPrimitiveType();
+      if (primitive.getOriginalType() != null) {
+        switch (columnDescriptor.getPrimitiveType().getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+          case BSON:
+            this.vec = new IcebergArrowVectors.VarcharArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
 
 Review comment:
   Should 10 be a `private static final` variable instead of hard-coded here? Looks like this is supposed to be the default average value size for varchars.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367727212
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
 
 Review comment:
   Can we clear the validity buffer at the start of each batch read and then only set the validity buffer and nullability holder when the value is defined?
   
   We may also be able to detect whether a column is mostly nulls by looking at the column stats. That way we could set all values to non-null and call this for every non-null value, depending on what is more likely for a column.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368105495
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
 
 Review comment:
   ```shouldAdjustToUTC``` is an instance level method. Not sure how to apply the check here. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367731129
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] decimalBytes = dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe();
+              byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+              System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP) This
 
 Review comment:
   Is this correct?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367730827
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
 
 Review comment:
   If this is because the values are in the dictionary, then we can build a dictionary decoder that produces byte buffers instead of binary. That should be simple.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367722355
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
 
 Review comment:
   Nevermind, looks like this is only used for dl, rl, and dictionary indices.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380413043
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,247 @@
+/*
+ * 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 org.apache.parquet.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.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public class BaseVectorizedParquetValuesReader extends ValuesReader {
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  MODE mode;
 
 Review comment:
   I prefer making it bit more restrictive. At this point, I expect the sub-classes to be included in this class's package only. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367724288
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
 
 Review comment:
   I think this name is slightly misleading. I expected it to be the vector capacity, but it looks like this is the number of values to read. Can we rename to clarify?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387192623
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java
 ##########
 @@ -23,26 +23,18 @@
 import java.io.IOException;
 import org.apache.parquet.CorruptDeltaByteArrays;
 import org.apache.parquet.bytes.ByteBufferInputStream;
-import org.apache.parquet.bytes.BytesInput;
-import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.Dictionary;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.ValuesType;
 import org.apache.parquet.column.page.DataPage;
 import org.apache.parquet.column.page.DataPageV1;
 import org.apache.parquet.column.page.DataPageV2;
 import org.apache.parquet.column.values.RequiresPreviousReader;
 import org.apache.parquet.column.values.ValuesReader;
-import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
 import org.apache.parquet.io.ParquetDecodingException;
 import org.apache.parquet.io.api.Binary;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class PageIterator<T> implements TripleIterator<T> {
-  private static final Logger LOG = LoggerFactory.getLogger(PageIterator.class);
 
+abstract class PageIterator<T> extends BasePageIterator implements TripleIterator<T> {
 
 Review comment:
   Why is this abstract?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367636484
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
 
 Review comment:
   This should check whether the `TimestampType` has `shouldAdjustToUTC` set. If set, then `"UTC"` is correct. If not set, then this should pass `null`. See https://github.com/apache/arrow/blob/master/format/Schema.fbs#L187-L197

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367648232
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/NullabilityHolder.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.iceberg.arrow.vectorized;
+
+public class NullabilityHolder {
+  private final boolean[] isNull;
+  private int numNulls;
+
+  public NullabilityHolder(int batchSize) {
+    this.isNull = new boolean[batchSize];
+  }
+
+  public void setNull(int idx) {
+    isNull[idx] = true;
+    numNulls++;
+  }
+
+  public boolean isNullAt(int idx) {
+    return isNull[idx];
 
 Review comment:
   All of the uses of `isNullAt` (at least, in this PR) convert boolean to 0 or 1 using a ternary operator. Should we store `isNull` as `byte[]` instead? Would that improve or hurt performance? Maybe there are uses elsewhere that make boolean the right choice here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r382857980
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
 
 Review comment:
   DEFAULT_BATCH_SIZE is used in the spark module.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367658467
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.isFixedLengthDecimal = false;
+    this.isVarWidthType = false;
+    this.isFixedWidthBinary = false;
+    this.isBooleanType = false;
+    this.isPaddedDecimal = false;
+    this.isIntType = false;
+    this.isLongType = false;
+    this.isFloatType = false;
+    this.isDoubleType = false;
+    this.vectorizedColumnIterator = null;
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      typeWidth = allocateFieldVector();
+    }
+    vec.setValueCount(0);
+    nullabilityHolder = new NullabilityHolder(batchSize);
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        if (isFixedLengthDecimal) {
+          vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+          ((IcebergDecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
 
 Review comment:
   Why does this use an unsafe cast, while fixed width below checks the vector class?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367721996
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
 
 Review comment:
   Is this correct for all types? Seems like this might be incorrect for variable-length or 8-byte types.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367666772
 
 

 ##########
 File path: build.gradle
 ##########
 @@ -337,6 +353,9 @@ project(':iceberg-spark-runtime') {
     relocate 'org.codehaus.jackson', 'org.apache.iceberg.shaded.org.apache.parquet.shaded.org.codehaus.jackson'
     relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc'
     relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift'
+    // relocate Arrow and related deps to shade Iceberg specific version
+    relocate 'io.netty.buffer', 'org.apache.iceberg.shaded.io.netty.buffer'
+    relocate 'org.apache.arrow', 'org.apache.iceberg.shaded.org.apache.arrow'
 
 Review comment:
   This will require updates to our binary LICENSE and NOTICE files.
   
   Do we know if this pulls in any transitive dependencies as well?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367641988
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
+        break;
+      case DATE:
+        arrowType = new ArrowType.Date(DateUnit.DAY);
+        break;
+      case STRUCT:
+        final StructType struct = field.type().asStructType();
+        arrowType = ArrowType.Struct.INSTANCE;
+
+        for (NestedField nested : struct.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case LIST:
+        final ListType listType = field.type().asListType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        for (NestedField nested : listType.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case MAP:
+        //Maps are represented as List<Struct<key, value>>
+        metadata = ImmutableMap.of(ORIGINAL_TYPE, MAP_TYPE);
+        final MapType mapType = field.type().asMapType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        final List<Field> entryFields = Lists.newArrayList(
+            convert(required(0, MAP_KEY, mapType.keyType())),
+            convert(optional(0, MAP_VALUE, mapType.valueType()))
+        );
+
+        final Field entry = new Field("",
+            new FieldType(true, new ArrowType.Struct(), null), entryFields);
 
 Review comment:
   I don't think that the key/value struct should be optional.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387235953
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java
 ##########
 @@ -23,26 +23,18 @@
 import java.io.IOException;
 import org.apache.parquet.CorruptDeltaByteArrays;
 import org.apache.parquet.bytes.ByteBufferInputStream;
-import org.apache.parquet.bytes.BytesInput;
-import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.Dictionary;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.ValuesType;
 import org.apache.parquet.column.page.DataPage;
 import org.apache.parquet.column.page.DataPageV1;
 import org.apache.parquet.column.page.DataPageV2;
 import org.apache.parquet.column.values.RequiresPreviousReader;
 import org.apache.parquet.column.values.ValuesReader;
-import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
 import org.apache.parquet.io.ParquetDecodingException;
 import org.apache.parquet.io.api.Binary;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class PageIterator<T> implements TripleIterator<T> {
-  private static final Logger LOG = LoggerFactory.getLogger(PageIterator.class);
 
+abstract class PageIterator<T> extends BasePageIterator implements TripleIterator<T> {
 
 Review comment:
   Oh, I forgot about that. Sounds good!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367670127
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java
 ##########
 @@ -0,0 +1,258 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Vectorized version of the ColumnIterator that reads column values in data pages of a column in a row group in a
+ * batched fashion.
+ */
+public class VectorizedColumnIterator {
+
+  private final ColumnDescriptor desc;
+  private final VectorizedPageIterator vectorizedPageIterator;
+
+  // state reset for each row group
+  private PageReader columnPageReader = null;
+  private long totalValuesCount = 0L;
+  private long valuesRead = 0L;
+  private long advanceNextPageCount = 0L;
+  private final int batchSize;
+
+  public VectorizedColumnIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
+    this.batchSize = batchSize;
+    this.vectorizedPageIterator = new VectorizedPageIterator(desc, writerVersion, batchSize);
+  }
+
+  public Dictionary setRowGroupInfo(PageReadStore store, boolean allPagesDictEncoded) {
+    this.columnPageReader = store.getPageReader(desc);
+    this.totalValuesCount = columnPageReader.getTotalValueCount();
 
 Review comment:
   The other iterator uses `store.getTotalValueCount`, not the one in `columnPageReader`. Are those guaranteed to be the same?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380403284
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java
 ##########
 @@ -0,0 +1,262 @@
+/*
+ * 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 org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Vectorized version of the ColumnIterator that reads column values in data pages of a column in a row group in a
+ * batched fashion.
+ */
+public class VectorizedColumnIterator {
+
+  private final ColumnDescriptor desc;
+  private final VectorizedPageIterator vectorizedPageIterator;
+
+  // state reset for each row group
+  private PageReader columnPageReader = null;
+  private long totalValuesCount = 0L;
+  private long valuesRead = 0L;
+  private long advanceNextPageCount = 0L;
+  private final int batchSize;
+
+  public VectorizedColumnIterator(ColumnDescriptor desc, String writerVersion, int batchSize,
+                                  boolean setArrowValidityVector) {
+    Preconditions.checkArgument(desc.getMaxRepetitionLevel() == 0,
+        "Only non-nested columns are supported for vectorized reads");
+    this.desc = desc;
+    this.batchSize = batchSize;
+    this.vectorizedPageIterator = new VectorizedPageIterator(desc, writerVersion, setArrowValidityVector);
+  }
+
+  public Dictionary setRowGroupInfo(PageReadStore store, boolean allPagesDictEncoded) {
+    this.columnPageReader = store.getPageReader(desc);
+    this.totalValuesCount = columnPageReader.getTotalValueCount();
+    this.valuesRead = 0L;
+    this.advanceNextPageCount = 0L;
+    this.vectorizedPageIterator.reset();
+    Dictionary dict = readDictionaryForColumn(store);
+    this.vectorizedPageIterator.setDictionaryForColumn(dict, allPagesDictEncoded);
+    advance();
+    return dict;
+  }
+
+  private void advance() {
+    if (valuesRead >= advanceNextPageCount) {
+      // A parquet page may be empty i.e. contains no values
+      while (!vectorizedPageIterator.hasNext()) {
+        DataPage page = columnPageReader.readPage();
+        if (page != null) {
+          vectorizedPageIterator.setPage(page);
+          this.advanceNextPageCount += vectorizedPageIterator.currentPageCount();
+        } else {
+          return;
+        }
+      }
+    }
+  }
+
+  public boolean hasNext() {
+    return valuesRead < totalValuesCount;
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP)
+   */
+  public void nextBatchIntegers(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) {
+    int rowsReadSoFar = 0;
+    while (rowsReadSoFar < batchSize && hasNext()) {
+      advance();
+      int rowsInThisBatch = vectorizedPageIterator.nextBatchIntegers(fieldVector, batchSize - rowsReadSoFar,
+          rowsReadSoFar, typeWidth, holder);
+      rowsReadSoFar += rowsInThisBatch;
+      this.valuesRead += rowsInThisBatch;
+      fieldVector.setValueCount(rowsReadSoFar);
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP)
+   */
+  public void nextBatchDictionaryIds(IntVector vector, NullabilityHolder holder) {
+    int rowsReadSoFar = 0;
+    while (rowsReadSoFar < batchSize && hasNext()) {
+      advance();
+      int rowsInThisBatch = vectorizedPageIterator.nextBatchDictionaryIds(vector, batchSize - rowsReadSoFar,
+          rowsReadSoFar, holder);
+      rowsReadSoFar += rowsInThisBatch;
+      this.valuesRead += rowsInThisBatch;
+      vector.setValueCount(rowsReadSoFar);
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP)
+   */
+  public void nextBatchLongs(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) {
+    int rowsReadSoFar = 0;
+    while (rowsReadSoFar < batchSize && hasNext()) {
+      advance();
+      int rowsInThisBatch = vectorizedPageIterator.nextBatchLongs(fieldVector, batchSize - rowsReadSoFar,
+          rowsReadSoFar, typeWidth, holder);
+      rowsReadSoFar += rowsInThisBatch;
+      this.valuesRead += rowsInThisBatch;
+      fieldVector.setValueCount(rowsReadSoFar);
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP)
+   */
+  public void nextBatchFloats(FieldVector fieldVector, int typeWidth, NullabilityHolder holder) {
+    int rowsReadSoFar = 0;
+    while (rowsReadSoFar < batchSize && hasNext()) {
+      advance();
+      int rowsInThisBatch = vectorizedPageIterator.nextBatchFloats(fieldVector, batchSize - rowsReadSoFar,
+          rowsReadSoFar, typeWidth, holder);
+      rowsReadSoFar += rowsInThisBatch;
+      this.valuesRead += rowsInThisBatch;
+      fieldVector.setValueCount(rowsReadSoFar);
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP)
 
 Review comment:
   These comments are all the same?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r382857679
 
 

 ##########
 File path: versions.lock
 ##########
 @@ -95,9 +96,9 @@ org.antlr:antlr4-runtime:4.7 (1 constraints: 7a0e125f)
 org.antlr:stringtemplate:3.2.1 (1 constraints: c10a3bc6)
 org.apache.ant:ant:1.9.1 (3 constraints: a721ed14)
 org.apache.ant:ant-launcher:1.9.1 (1 constraints: 69082485)
-org.apache.arrow:arrow-format:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-memory:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-vector:0.10.0 (1 constraints: e90c9734)
+org.apache.arrow:arrow-format:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-memory:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-vector:0.14.1 (2 constraints: 2012a545)
 
 Review comment:
   Unfortunately in 0.15.1, they made vectors like DecimalVector, etc. final which prevents overriding the isSet() method to use NullabilityHolder. So going to stick to 0.14.1 for now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367631627
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
 
 Review comment:
   Nit: checkstyle doesn't usually like static imports like this, so elsewhere we converted these to use `NestedField.optional` in the code. We should use the same style here.
   
   Also, is checkstyle turned on for iceberg-arrow?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r382849506
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java
 ##########
 @@ -0,0 +1,258 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Vectorized version of the ColumnIterator that reads column values in data pages of a column in a row group in a
+ * batched fashion.
+ */
+public class VectorizedColumnIterator {
+
+  private final ColumnDescriptor desc;
+  private final VectorizedPageIterator vectorizedPageIterator;
+
+  // state reset for each row group
+  private PageReader columnPageReader = null;
+  private long totalValuesCount = 0L;
+  private long valuesRead = 0L;
+  private long advanceNextPageCount = 0L;
+  private final int batchSize;
+
+  public VectorizedColumnIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
+    this.batchSize = batchSize;
+    this.vectorizedPageIterator = new VectorizedPageIterator(desc, writerVersion, batchSize);
+  }
+
+  public Dictionary setRowGroupInfo(PageReadStore store, boolean allPagesDictEncoded) {
 
 Review comment:
   We should do this in a follow-up.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367653224
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorHolder.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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;
+
+import javax.annotation.Nullable;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+
+/**
+ * Container class for holding the Arrow vector holding a batch of values along with other state needed for reading
+ * values out of it.
+ */
+public class VectorHolder {
+  private final ColumnDescriptor columnDescriptor;
+  private final FieldVector vector;
+  private final boolean isDictionaryEncoded;
+
+  @Nullable
+  private final Dictionary dictionary;
+  private final NullabilityHolder nullabilityHolder;
+
+  public static final VectorHolder NULL_VECTOR_HOLDER = new VectorHolder(null, null, false, null, null);
+
+  public VectorHolder(
+      ColumnDescriptor columnDescriptor,
+      FieldVector vector,
+      boolean isDictionaryEncoded,
+      Dictionary dictionary,
+      NullabilityHolder holder) {
+    this.columnDescriptor = columnDescriptor;
+    this.vector = vector;
+    this.isDictionaryEncoded = isDictionaryEncoded;
+    this.dictionary = dictionary;
+    this.nullabilityHolder = holder;
+  }
+
+  public ColumnDescriptor getDescriptor() {
 
 Review comment:
   Style: We don't use `get` in getter methods. As a verb, it doesn't add any clarity to what is happening, in contrast to verbs like `fetch` or `aggregate` that tell you some expensive action is occurring. It also makes expressions longer and is awkward when used from languages like Scala that don't use this convention.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380868211
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java
 ##########
 @@ -0,0 +1,202 @@
+/*
+ * 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.parquet;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class BasePageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(BasePageIterator.class);
+
+  protected final ColumnDescriptor desc;
+  protected final String writerVersion;
+
+  // iterator state
+  protected boolean hasNext = false;
+  protected int triplesRead = 0;
+  protected int currentDL = 0;
+  protected int currentRL = 0;
+
+  // page bookkeeping
+  protected Dictionary dictionary = null;
+  protected DataPage page = null;
+  protected int triplesCount = 0;
+  protected Encoding valueEncoding = null;
+  protected IntIterator definitionLevels = null;
+  protected IntIterator repetitionLevels = null;
+  protected ValuesReader vectorizedDefinitionLevelReader = null;
+  protected ValuesReader values = null;
+
+  protected BasePageIterator(ColumnDescriptor descriptor, String writerVersion) {
+    this.desc = descriptor;
+    this.writerVersion = writerVersion;
+  }
+
+  protected abstract void reset();
+
+  protected abstract boolean supportsVectorizedReads();
+
+  protected abstract IntIterator newNonVectorizedDefinitionLevelReader(ValuesReader dlReader);
+
+  protected abstract ValuesReader newVectorizedDefinitionLevelReader(ColumnDescriptor descriptor);
+
+  protected abstract void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount);
+
+  public void setPage(DataPage page) {
+    Preconditions.checkNotNull(page, "Cannot read from null page");
+    this.page = page;
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    this.hasNext = triplesRead < triplesCount;
+  }
+
+  protected void initFromPage(DataPageV1 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    ValuesReader dlReader = null;
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      dlReader = initPage.getDlEncoding().getValuesReader(desc, ValuesType.DEFINITION_LEVEL);
+      this.definitionLevels = newNonVectorizedDefinitionLevelReader(dlReader);
+    }
+    ValuesReader rlReader = initPage.getRlEncoding().getValuesReader(desc, ValuesType.REPETITION_LEVEL);
+    this.repetitionLevels = new PageIterator.ValuesReaderIntIterator(rlReader);
+    try {
+      BytesInput bytes = initPage.getBytes();
+      LOG.debug("page size {} bytes and {} records", bytes.size(), triplesCount);
+      LOG.debug("reading repetition levels at 0");
+      ByteBufferInputStream in = bytes.toInputStream();
+      rlReader.initFromPage(triplesCount, in);
+      LOG.debug("reading definition levels at {}", in.position());
+      if (supportsVectorizedReads()) {
+        this.vectorizedDefinitionLevelReader.initFromPage(triplesCount, in);
+      } else {
+        dlReader.initFromPage(triplesCount, in);
+      }
+      LOG.debug("reading data at {}", in.position());
+      initDataReader(initPage.getValueEncoding(), in, initPage.getValueCount());
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  protected void initFromPage(DataPageV2 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    this.repetitionLevels = newRLEIterator(desc.getMaxRepetitionLevel(), initPage.getRepetitionLevels());
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      this.definitionLevels = newRLEIterator(desc.getMaxDefinitionLevel(), initPage.getDefinitionLevels());
+    }
+    LOG.debug("page data size {} bytes and {} records", initPage.getData().size(), triplesCount);
+    try {
+      initDataReader(initPage.getDataEncoding(), initPage.getData().toInputStream(), triplesCount);
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
+    try {
+      if (maxLevel == 0) {
+        return new NullIntIterator();
+      }
+      return new RLEIntIterator(
+          new RunLengthBitPackingHybridDecoder(
+              BytesUtils.getWidthFromMaxInt(maxLevel),
+              bytes.toInputStream()));
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read levels in page for col " + desc, e);
+    }
+  }
+
+  public void setDictionary(Dictionary dict) {
+    this.dictionary = dict;
+  }
+
+  protected abstract static class IntIterator {
+    abstract int nextInt();
+  }
+
+  static class ValuesReaderIntIterator extends IntIterator {
 
 Review comment:
   `ValuesReaderIntIterator` is only used in the non vectorized version `PageIterator`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380820584
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java
 ##########
 @@ -0,0 +1,202 @@
+/*
+ * 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.parquet;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class BasePageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(BasePageIterator.class);
+
+  protected final ColumnDescriptor desc;
+  protected final String writerVersion;
+
+  // iterator state
+  protected boolean hasNext = false;
+  protected int triplesRead = 0;
+  protected int currentDL = 0;
+  protected int currentRL = 0;
+
+  // page bookkeeping
+  protected Dictionary dictionary = null;
+  protected DataPage page = null;
+  protected int triplesCount = 0;
+  protected Encoding valueEncoding = null;
+  protected IntIterator definitionLevels = null;
+  protected IntIterator repetitionLevels = null;
+  protected ValuesReader vectorizedDefinitionLevelReader = null;
+  protected ValuesReader values = null;
+
+  protected BasePageIterator(ColumnDescriptor descriptor, String writerVersion) {
+    this.desc = descriptor;
+    this.writerVersion = writerVersion;
+  }
+
+  protected abstract void reset();
+
+  protected abstract boolean supportsVectorizedReads();
+
+  protected abstract IntIterator newNonVectorizedDefinitionLevelReader(ValuesReader dlReader);
+
+  protected abstract ValuesReader newVectorizedDefinitionLevelReader(ColumnDescriptor descriptor);
+
+  protected abstract void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount);
+
+  public void setPage(DataPage page) {
+    Preconditions.checkNotNull(page, "Cannot read from null page");
+    this.page = page;
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    this.hasNext = triplesRead < triplesCount;
+  }
+
+  protected void initFromPage(DataPageV1 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    ValuesReader dlReader = null;
+    if (supportsVectorizedReads()) {
 
 Review comment:
   Couldn't this create a vectorized definition level reader either way? That still exposes `nextInteger()`, which is all the non-vectorized version needs to call for an `IntIterator`. Then you could always use the same DL reader and the subclasses can choose how to wrap it (non-vectorized) or call its methods (vectorized).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387186742
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1005 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.column.Dictionary;
+
+public final class VectorizedParquetValuesReader extends BaseVectorizedParquetValuesReader {
 
 Review comment:
   Can you rename this to `VectorizedParquetDefinitionLevelReader`? I'd like to capture that this class is a `ValuesReader` that reads definition levels and then uses another reader for values.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387373836
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java
 ##########
 @@ -0,0 +1,202 @@
+/*
+ * 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.parquet;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class BasePageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(BasePageIterator.class);
+
+  protected final ColumnDescriptor desc;
+  protected final String writerVersion;
+
+  // iterator state
+  protected boolean hasNext = false;
+  protected int triplesRead = 0;
+  protected int currentDL = 0;
+  protected int currentRL = 0;
+
+  // page bookkeeping
+  protected Dictionary dictionary = null;
+  protected DataPage page = null;
+  protected int triplesCount = 0;
+  protected Encoding valueEncoding = null;
+  protected IntIterator definitionLevels = null;
+  protected IntIterator repetitionLevels = null;
+  protected ValuesReader vectorizedDefinitionLevelReader = null;
+  protected ValuesReader values = null;
+
+  protected BasePageIterator(ColumnDescriptor descriptor, String writerVersion) {
+    this.desc = descriptor;
+    this.writerVersion = writerVersion;
+  }
+
+  protected abstract void reset();
+
+  protected abstract boolean supportsVectorizedReads();
+
+  protected abstract IntIterator newNonVectorizedDefinitionLevelReader(ValuesReader dlReader);
+
+  protected abstract ValuesReader newVectorizedDefinitionLevelReader(ColumnDescriptor descriptor);
+
+  protected abstract void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount);
+
+  public void setPage(DataPage page) {
+    Preconditions.checkNotNull(page, "Cannot read from null page");
+    this.page = page;
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    this.hasNext = triplesRead < triplesCount;
+  }
+
+  protected void initFromPage(DataPageV1 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    ValuesReader dlReader = null;
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      dlReader = initPage.getDlEncoding().getValuesReader(desc, ValuesType.DEFINITION_LEVEL);
+      this.definitionLevels = newNonVectorizedDefinitionLevelReader(dlReader);
+    }
+    ValuesReader rlReader = initPage.getRlEncoding().getValuesReader(desc, ValuesType.REPETITION_LEVEL);
+    this.repetitionLevels = new PageIterator.ValuesReaderIntIterator(rlReader);
+    try {
+      BytesInput bytes = initPage.getBytes();
+      LOG.debug("page size {} bytes and {} records", bytes.size(), triplesCount);
+      LOG.debug("reading repetition levels at 0");
+      ByteBufferInputStream in = bytes.toInputStream();
+      rlReader.initFromPage(triplesCount, in);
+      LOG.debug("reading definition levels at {}", in.position());
+      if (supportsVectorizedReads()) {
+        this.vectorizedDefinitionLevelReader.initFromPage(triplesCount, in);
+      } else {
+        dlReader.initFromPage(triplesCount, in);
+      }
+      LOG.debug("reading data at {}", in.position());
+      initDataReader(initPage.getValueEncoding(), in, initPage.getValueCount());
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  protected void initFromPage(DataPageV2 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    this.repetitionLevels = newRLEIterator(desc.getMaxRepetitionLevel(), initPage.getRepetitionLevels());
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      this.definitionLevels = newRLEIterator(desc.getMaxDefinitionLevel(), initPage.getDefinitionLevels());
+    }
+    LOG.debug("page data size {} bytes and {} records", initPage.getData().size(), triplesCount);
+    try {
+      initDataReader(initPage.getDataEncoding(), initPage.getData().toInputStream(), triplesCount);
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
+    try {
+      if (maxLevel == 0) {
+        return new NullIntIterator();
+      }
+      return new RLEIntIterator(
+          new RunLengthBitPackingHybridDecoder(
+              BytesUtils.getWidthFromMaxInt(maxLevel),
+              bytes.toInputStream()));
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read levels in page for col " + desc, e);
+    }
+  }
+
+  public void setDictionary(Dictionary dict) {
+    this.dictionary = dict;
+  }
+
+  protected abstract static class IntIterator {
+    abstract int nextInt();
+  }
+
+  static class ValuesReaderIntIterator extends IntIterator {
 
 Review comment:
   I misspoke earlier. The `ValuesReaderIntIterator` is used in BasePageIterator as well.
   `
       this.repetitionLevels = new ValuesReaderIntIterator(rlReader);
   `
   which is used by both the code paths.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380402565
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,247 @@
+/*
+ * 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 org.apache.parquet.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.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public class BaseVectorizedParquetValuesReader extends ValuesReader {
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  MODE mode;
+  int currentCount;
+  int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  int[] packedValuesBuffer = new int[16];
+  int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  final int maxDefLevel;
+
+  final boolean setArrowValidityVector;
+
+  public BaseVectorizedParquetValuesReader(int maxDefLevel, boolean setValidityVector) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+    this.setArrowValidityVector = setValidityVector;
+  }
+
+  public BaseVectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel,
+      boolean setValidityVector) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    this.setArrowValidityVector = setValidityVector;
+    init(bitWidth);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Non-supported bytesWidth: " + bytesWidth);
+  }
+
+  /**
+   * Reads the next group.
+   */
+  void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
 
 Review comment:
   Is this `readInteger` call correct? If it isn't correct for all types, it would be better to throw `UnsupportedOperationException` and replace uses with a `skipInteger()` call.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on issue #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on issue #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#issuecomment-594083320
 
 
   @samarthjain, only a few minor things left. Could you remove all of the patch files from the PR?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380404000
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java
 ##########
 @@ -0,0 +1,408 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.Dictionary;
+
+public class VectorizedDictionaryEncodedParquetValuesReader extends BaseVectorizedParquetValuesReader {
+
+  public VectorizedDictionaryEncodedParquetValuesReader(int maxDefLevel, boolean setValidityVector) {
+    super(maxDefLevel, setValidityVector);
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  void readBatchOfDictionaryIds(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead,
 
 Review comment:
   Also, `numValsInVector` is really an offset. Can we use a better name, like `startOffset`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367728699
 
 

 ##########
 File path: arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java
 ##########
 @@ -0,0 +1,113 @@
+/*
+ * 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;
+
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.BooleanType;
+import org.apache.iceberg.types.Types.DateType;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.iceberg.types.Types.TimestampType;
+import org.junit.Test;
+
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Bool;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Date;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.FloatingPoint;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Int;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.List;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Timestamp;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+
+public class ArrowSchemaUtilTest {
 
 Review comment:
   I think there should be a few more tests for this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380412505
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,247 @@
+/*
+ * 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 org.apache.parquet.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.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public class BaseVectorizedParquetValuesReader extends ValuesReader {
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  MODE mode;
+  int currentCount;
+  int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  int[] packedValuesBuffer = new int[16];
+  int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  final int maxDefLevel;
+
+  final boolean setArrowValidityVector;
+
+  public BaseVectorizedParquetValuesReader(int maxDefLevel, boolean setValidityVector) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+    this.setArrowValidityVector = setValidityVector;
+  }
+
+  public BaseVectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel,
+      boolean setValidityVector) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    this.setArrowValidityVector = setValidityVector;
+    init(bitWidth);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Non-supported bytesWidth: " + bytesWidth);
+  }
+
+  /**
+   * Reads the next group.
+   */
+  void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
 
 Review comment:
   The readInteger() call figures out how the next set of definition levels are encoded (RLE vs PACKED) by reading the next integer. It doesn't have to do with the actual record value. 
    

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367659147
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.isFixedLengthDecimal = false;
+    this.isVarWidthType = false;
+    this.isFixedWidthBinary = false;
+    this.isBooleanType = false;
+    this.isPaddedDecimal = false;
+    this.isIntType = false;
+    this.isLongType = false;
+    this.isFloatType = false;
+    this.isDoubleType = false;
+    this.vectorizedColumnIterator = null;
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      typeWidth = allocateFieldVector();
+    }
+    vec.setValueCount(0);
+    nullabilityHolder = new NullabilityHolder(batchSize);
 
 Review comment:
   I think I would switch this around: if the vector has a `NullabilityHolder`, then use it. Then you get one instance check here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367733092
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] decimalBytes = dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe();
+              byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+              System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP) This
+   * method reads batches of bytes from Parquet and writes them into the data buffer underneath the Arrow vector. It
+   * appropriately sets the validity buffer in the Arrow vector.
+   */
+  public void readBatchVarWidth(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setVarWidthBinaryValue(FieldVector vector, ValuesAsBytesReader valuesReader, int bufferIdx) {
+    int len = valuesReader.readInteger();
+    ByteBuffer buffer = valuesReader.getBuffer(len);
+    // Calling setValueLengthSafe takes care of allocating a larger buffer if
+    // running out of space.
+    ((BaseVariableWidthVector) vector).setValueLengthSafe(bufferIdx, len);
+    // It is possible that the data buffer was reallocated. So it is important to
+    // not cache the data buffer reference but instead use vector.getDataBuffer().
+    vector.getDataBuffer().writeBytes(buffer.array(), buffer.position(), buffer.limit() - buffer.position());
+    // Similarly, we need to get the latest reference to the validity buffer as well
+    // since reallocation changes reference of the validity buffers as well.
+    BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedVarWidth(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedVarWidthBinaryInternal(vector, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((BaseVariableWidthVector) vector).setSafe(
+                  idx,
+                  dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedVarWidthBinaryInternal(
+      FieldVector vector,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            ((BaseVariableWidthVector) vector).setSafe(idx, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            ((BaseVariableWidthVector) vector).setSafe(
+                idx,
+                dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfIntLongBackedDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth);
+              vector.getDataBuffer().setBytes(bufferIdx * DecimalVector.TYPE_WIDTH, byteArray);
+              BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth);
+              vector.getDataBuffer().setBytes(bufferIdx * DecimalVector.TYPE_WIDTH, byteArray);
+              BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntLongBackedDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntLongBackedDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((DecimalVector) vector).set(
+                  idx,
+                  typeWidth == Integer.BYTES ?
+                      dict.decodeToInt(valuesReader.readInteger())
+                      : dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntLongBackedDecimalsInternal(
+      FieldVector vector,
+      final int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            ((DecimalVector) vector).set(
+                idx,
+                typeWidth == Integer.BYTES ? dict.decodeToInt(currentValue) : dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            ((DecimalVector) vector).set(
+                idx,
+                typeWidth == Integer.BYTES ?
+                    dict.decodeToInt(currentValue)
+                    : dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfBooleans(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              ((BitVector) vector).setSafe(bufferIdx, valuesReader.readBoolean() ? 1 : 0);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((BitVector) vector).setSafe(bufferIdx, valuesReader.readBoolean() ? 1 : 0);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setBinaryInVector(
+      VarBinaryVector vector,
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx) {
+    byte[] byteArray = new byte[typeWidth];
+    valuesReader.getBuffer(typeWidth).get(byteArray);
+    vector.setSafe(bufferIdx, byteArray);
 
 Review comment:
   We use this to trigger reallocation if necessary?
   
   Is there a `setSafe` that accepts a `ByteBuffer` to avoid this copy? If not, we should contribute one back to Arrow.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367732800
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] decimalBytes = dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe();
+              byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+              System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP) This
+   * method reads batches of bytes from Parquet and writes them into the data buffer underneath the Arrow vector. It
+   * appropriately sets the validity buffer in the Arrow vector.
+   */
+  public void readBatchVarWidth(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setVarWidthBinaryValue(FieldVector vector, ValuesAsBytesReader valuesReader, int bufferIdx) {
+    int len = valuesReader.readInteger();
+    ByteBuffer buffer = valuesReader.getBuffer(len);
+    // Calling setValueLengthSafe takes care of allocating a larger buffer if
+    // running out of space.
+    ((BaseVariableWidthVector) vector).setValueLengthSafe(bufferIdx, len);
+    // It is possible that the data buffer was reallocated. So it is important to
+    // not cache the data buffer reference but instead use vector.getDataBuffer().
+    vector.getDataBuffer().writeBytes(buffer.array(), buffer.position(), buffer.limit() - buffer.position());
+    // Similarly, we need to get the latest reference to the validity buffer as well
+    // since reallocation changes reference of the validity buffers as well.
+    BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedVarWidth(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedVarWidthBinaryInternal(vector, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((BaseVariableWidthVector) vector).setSafe(
+                  idx,
+                  dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedVarWidthBinaryInternal(
+      FieldVector vector,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            ((BaseVariableWidthVector) vector).setSafe(idx, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            ((BaseVariableWidthVector) vector).setSafe(
+                idx,
+                dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfIntLongBackedDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth);
+              vector.getDataBuffer().setBytes(bufferIdx * DecimalVector.TYPE_WIDTH, byteArray);
+              BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth);
+              vector.getDataBuffer().setBytes(bufferIdx * DecimalVector.TYPE_WIDTH, byteArray);
+              BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntLongBackedDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntLongBackedDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((DecimalVector) vector).set(
+                  idx,
+                  typeWidth == Integer.BYTES ?
+                      dict.decodeToInt(valuesReader.readInteger())
+                      : dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntLongBackedDecimalsInternal(
+      FieldVector vector,
+      final int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            ((DecimalVector) vector).set(
+                idx,
+                typeWidth == Integer.BYTES ? dict.decodeToInt(currentValue) : dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            ((DecimalVector) vector).set(
+                idx,
+                typeWidth == Integer.BYTES ?
+                    dict.decodeToInt(currentValue)
+                    : dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfBooleans(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              ((BitVector) vector).setSafe(bufferIdx, valuesReader.readBoolean() ? 1 : 0);
 
 Review comment:
   Is there any benefit to returning 0 or 1 directly? We could use `readBooleanAsInt` and use a shift to get either 1 or 0. Then we wouldn't need to branch here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367722888
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
 
 Review comment:
   Giving a bit more context about what violated assumptions here is usually good. I'd probably update it to this:
   
   ```java
   throw new RuntimeException("Unsupported hybrid decode mode: " + mode);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367640394
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
+        break;
+      case DATE:
+        arrowType = new ArrowType.Date(DateUnit.DAY);
+        break;
+      case STRUCT:
+        final StructType struct = field.type().asStructType();
+        arrowType = ArrowType.Struct.INSTANCE;
+
+        for (NestedField nested : struct.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case LIST:
+        final ListType listType = field.type().asListType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        for (NestedField nested : listType.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case MAP:
+        //Maps are represented as List<Struct<key, value>>
+        metadata = ImmutableMap.of(ORIGINAL_TYPE, MAP_TYPE);
+        final MapType mapType = field.type().asMapType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        final List<Field> entryFields = Lists.newArrayList(
+            convert(required(0, MAP_KEY, mapType.keyType())),
+            convert(optional(0, MAP_VALUE, mapType.valueType()))
+        );
+
+        final Field entry = new Field("",
+            new FieldType(true, new ArrowType.Struct(), null), entryFields);
+        children.add(entry);
+        break;
+      default: throw new UnsupportedOperationException("Unsupported field type: " + field);
 
 Review comment:
   Please add a newline before the `throw`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r369881356
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
 
 Review comment:
   The ```setNextNValuesInVector``` method checks ```currentValue```
   
   ```
   private void setNextNValuesInVector(
         int typeWidth, NullabilityHolder nullabilityHolder,
         ValuesAsBytesReader valuesReader, int bufferIdx, FieldVector vector, int numValues) {
       ArrowBuf validityBuffer = vector.getValidityBuffer();
       int validityBufferIdx = bufferIdx;
       if (currentValue == maxDefLevel) {
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368103333
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
 
 Review comment:
   On editors like Intellij, it tells me what the parameter name is. 
   Example:
   new ArrowType.Int(Integer.SIZE, isSigned: true);
   
   Not sure I agree with having to add a comment for every parameter. Method signature should be enough, no?
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367719342
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,565 @@
+/*
+ * 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 com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class VectorizedPageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(VectorizedPageIterator.class);
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
+    this.writerVersion = writerVersion;
+  }
+
+  private final ColumnDescriptor desc;
+  private final String writerVersion;
+
+  // iterator state
+  private boolean hasNext = false;
+  private int triplesRead = 0;
+
+  // page bookkeeping
+  private Dictionary dictionary = null;
+  private DataPage page = null;
+  private int triplesCount = 0;
+
+  // Needed once we add support for complex types. Unused for now.
+  private IntIterator repetitionLevels = null;
+  private int currentRL = 0;
+
+  private VectorizedParquetValuesReader definitionLevelReader;
+  private boolean eagerDecodeDictionary;
+  private ValuesAsBytesReader plainValuesReader = null;
+  private VectorizedParquetValuesReader dictionaryEncodedValuesReader = null;
+  private boolean allPagesDictEncoded;
+
+  public void setPage(DataPage dataPage) {
+    this.page = Preconditions.checkNotNull(dataPage, "Cannot read from null page");
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    advance();
+  }
+
+  // Dictionary is set per row group
+  public void setDictionaryForColumn(Dictionary dict, boolean allDictEncoded) {
+    this.dictionary = dict;
+    this.allPagesDictEncoded = allDictEncoded;
+  }
+
+  public void reset() {
+    this.page = null;
+    this.triplesCount = 0;
+    this.triplesRead = 0;
+    this.repetitionLevels = null;
+    this.plainValuesReader = null;
+    this.definitionLevelReader = null;
+    this.hasNext = false;
+  }
+
+  public int currentPageCount() {
+    return triplesCount;
+  }
+
+  public boolean hasNext() {
+    return hasNext;
+  }
+
+  /**
+   * Method for reading a batch of dictionary ids from the dicitonary encoded data pages. Like definition levels,
+   * dictionary ids in Parquet are RLE/bin-packed encoded as well.
+   */
+  public int nextBatchDictionaryIds(
+      final IntVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    definitionLevelReader.readBatchOfDictionaryIds(
+        vector,
+        numValsInVector,
+        actualBatchSize,
+        holder,
+        dictionaryEncodedValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT32 data type
+   */
+  public int nextBatchIntegers(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT64 data type
+   */
+  public int nextBatchLongs(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of FLOAT data type.
+   */
+  public int nextBatchFloats(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of DOUBLE data type
+   */
+  public int nextBatchDoubles(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  private int getActualBatchSize(int expectedBatchSize) {
+    return Math.min(expectedBatchSize, triplesCount - triplesRead);
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by INT32 and INT64 parquet data types. Since Arrow stores all
+   * decimals in 16 bytes, byte arrays are appropriately padded before being written to Arrow data buffers.
+   */
+  public int nextBatchIntLongBackedDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedIntLongBackedDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfIntLongBackedDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by fixed length byte array parquet data type. Arrow stores all
+   * decimals in 16 bytes. This method provides the necessary padding to the decimals read. Moreover, Arrow interprets
+   * the decimals in Arrow buffer as little endian. Parquet stores fixed length decimals as big endian. So, this method
+   * uses {@link DecimalVector#setBigEndian(int, byte[])} method so that the data in Arrow vector is indeed little
+   * endian.
+   */
+  public int nextBatchFixedLengthDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of variable width data type (ENUM, JSON, UTF8, BSON).
+   */
+  public int nextBatchVarWidthType(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of fixed width binary type (e.g. BYTE[7]). Spark does not support fixed width binary
+   * data type. To work around this limitation, the data is read as fixed width binary from parquet and stored in a
+   * {@link VarBinaryVector} in Arrow.
+   */
+  public int nextBatchFixedWidthBinary(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      definitionLevelReader.readBatchOfDictionaryEncodedFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      definitionLevelReader.readBatchOfFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of booleans.
+   */
+  public int nextBatchBoolean(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    definitionLevelReader.readBatchOfBooleans(vector, numValsInVector, actualBatchSize,
+        nullabilityHolder, plainValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  private void advance() {
+    if (triplesRead < triplesCount) {
+      this.hasNext = true;
+    } else {
+      this.hasNext = false;
+    }
+  }
+
+  private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) {
+    ValuesReader previousReader = plainValuesReader;
+    this.eagerDecodeDictionary = dataEncoding.usesDictionary() && dictionary != null && !allPagesDictEncoded;
+    if (dataEncoding.usesDictionary()) {
+      if (dictionary == null) {
+        throw new ParquetDecodingException(
+            "could not read page in col " + desc + " as the dictionary was missing for encoding " + dataEncoding);
+      }
+      try {
+        dictionaryEncodedValuesReader =
+            new VectorizedParquetValuesReader(desc.getMaxDefinitionLevel());
+        dictionaryEncodedValuesReader.initFromPage(valueCount, in);
+      } catch (IOException e) {
+        throw new ParquetDecodingException("could not read page in col " + desc, e);
+      }
+    } else {
+      plainValuesReader = new ValuesAsBytesReader();
+      plainValuesReader.initFromPage(valueCount, in);
+    }
+    if (CorruptDeltaByteArrays.requiresSequentialReads(writerVersion, dataEncoding) &&
+        previousReader != null && previousReader instanceof RequiresPreviousReader) {
+      // previous reader can only be set if reading sequentially
+      ((RequiresPreviousReader) plainValuesReader).setPreviousReader(previousReader);
+    }
+  }
+
+  private void initFromPage(DataPageV1 dataPageV1) {
+    this.triplesCount = dataPageV1.getValueCount();
+    ValuesReader rlReader = dataPageV1.getRlEncoding().getValuesReader(desc, ValuesType.REPETITION_LEVEL);
+    ValuesReader dlReader;
 
 Review comment:
   I don't think `dlReader` is needed any more?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367672096
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java
 ##########
 @@ -0,0 +1,258 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Vectorized version of the ColumnIterator that reads column values in data pages of a column in a row group in a
+ * batched fashion.
+ */
+public class VectorizedColumnIterator {
+
+  private final ColumnDescriptor desc;
+  private final VectorizedPageIterator vectorizedPageIterator;
+
+  // state reset for each row group
+  private PageReader columnPageReader = null;
+  private long totalValuesCount = 0L;
+  private long valuesRead = 0L;
+  private long advanceNextPageCount = 0L;
+  private final int batchSize;
+
+  public VectorizedColumnIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
+    this.batchSize = batchSize;
+    this.vectorizedPageIterator = new VectorizedPageIterator(desc, writerVersion, batchSize);
+  }
+
+  public Dictionary setRowGroupInfo(PageReadStore store, boolean allPagesDictEncoded) {
+    this.columnPageReader = store.getPageReader(desc);
+    this.totalValuesCount = columnPageReader.getTotalValueCount();
+    this.valuesRead = 0L;
+    this.advanceNextPageCount = 0L;
+    this.vectorizedPageIterator.reset();
+    Dictionary dict = readDictionaryForColumn(store);
 
 Review comment:
   This `readDictionaryForColumn` uses `desc` and calls `getPageReader` that is called just above to initialize `columnPageReader`. I think it would be better to use the same `readDictionary` static method that is used by the non-vectorized path. For one thing, it would eliminate the duplicate call to `getPageReader` because you'd pass `columnPageReader` in, and it would also get rid of the unknown use of `desc` because it is static. And we'd not need to maintain two copies of the method that do pretty much the same thing.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380401584
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
 
 Review comment:
   This is used to initialize `typeWidth`, but it looks like all cases where `typeWidth` is passed during read expect it to be set correctly instead of -1. If that's the case, then let's make `typeWidth` an `Integer` and set this to `null`. That way, using it incorrectly at least results in a `NullPointerException`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368132505
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
 
 Review comment:
   You just need to cast `field.type()` to `TimestampType` and then call `shouldAdjustToUTC`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380824253
 
 

 ##########
 File path: versions.lock
 ##########
 @@ -95,9 +96,9 @@ org.antlr:antlr4-runtime:4.7 (1 constraints: 7a0e125f)
 org.antlr:stringtemplate:3.2.1 (1 constraints: c10a3bc6)
 org.apache.ant:ant:1.9.1 (3 constraints: a721ed14)
 org.apache.ant:ant-launcher:1.9.1 (1 constraints: 69082485)
-org.apache.arrow:arrow-format:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-memory:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-vector:0.10.0 (1 constraints: e90c9734)
+org.apache.arrow:arrow-format:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-memory:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-vector:0.14.1 (2 constraints: 2012a545)
 
 Review comment:
   @samarthjain, did you see this question?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367668075
 
 

 ##########
 File path: arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java
 ##########
 @@ -0,0 +1,113 @@
+/*
+ * 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;
+
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.BooleanType;
+import org.apache.iceberg.types.Types.DateType;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.iceberg.types.Types.TimestampType;
+import org.junit.Test;
+
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Bool;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Date;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.FloatingPoint;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Int;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.List;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Timestamp;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+
+public class ArrowSchemaUtilTest {
+
+  @Test
+  public void convertPrimitive() {
+    Schema iceberg = new Schema(
+        optional(0, "i", Types.IntegerType.get()),
+        optional(1, "b", BooleanType.get()),
+        required(2, "d", DoubleType.get()),
+        required(3, "s", StringType.get()),
+        optional(4, "d2", DateType.get()),
+        optional(5, "ts", TimestampType.withoutZone())
+    );
+    org.apache.arrow.vector.types.pojo.Schema arrow = ArrowSchemaUtil.convert(iceberg);
+    validate(iceberg, arrow);
+  }
+
+  @Test
+  public void convertComplex() {
+    Schema iceberg = new Schema(
+        optional(0, "m", MapType.ofOptional(
+            1, 2, StringType.get(),
+            LongType.get())
+        ),
+        required(3, "m2", MapType.ofOptional(
+            4, 5, StringType.get(),
+            ListType.ofOptional(6, TimestampType.withoutZone()))
+        )
+    );
+    org.apache.arrow.vector.types.pojo.Schema arrow = ArrowSchemaUtil.convert(iceberg);
+    assertEquals(iceberg.columns().size(), arrow.getFields().size());
+  }
+
+  private void validate(Schema iceberg, org.apache.arrow.vector.types.pojo.Schema arrow) {
+    assertEquals(iceberg.columns().size(), arrow.getFields().size());
+
+    for (Types.NestedField nf : iceberg.columns()) {
+      Field field = arrow.findField(nf.name());
+      assertNotNull("Missing filed: " + nf, field);
+
+      validate(nf.type(), field.getType());
+    }
+  }
+
+  private void validate(Type iceberg, ArrowType arrow) {
+    switch (iceberg.typeId()) {
+      case BOOLEAN: assertEquals(Bool, arrow.getTypeID());
 
 Review comment:
   Can we move the asserts to a separate line? And also use `Assert.assertEquals` to match our no-static-method-import style.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380817330
 
 

 ##########
 File path: build.gradle
 ##########
 @@ -228,6 +228,21 @@ project(':iceberg-parquet') {
   }
 }
 
+project(':iceberg-arrow') {
+  dependencies {
+    compile project(':iceberg-api')
+    compile project(':iceberg-parquet')
+
+    compile("org.apache.arrow:arrow-vector") {
+      exclude group: 'io.netty', module: 'netty-buffer'
+      exclude group: 'io.netty', module: 'netty-common'
 
 Review comment:
   @samarthjain, did you answer this already? I don't remember.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380401203
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
 
 Review comment:
   Do these need to be public?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380400940
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private ReadType readType;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize,
+      boolean setArrowValidityVector) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize, setArrowValidityVector);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.vectorizedColumnIterator = null;
+  }
+
+  private enum ReadType {
+    FIXED_LENGTH_DECIMAL, INT_LONG_BACKED_DECIMAL, VARCHAR, VARBINARY, FIXED_WIDTH_BINARY,
+    BOOLEAN, INT, LONG, FLOAT, DOUBLE
+  }
+
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      allocateFieldVector();
+      nullabilityHolder = new NullabilityHolder(batchSize);
+    } else {
+      vec.setValueCount(0);
+      nullabilityHolder.reset();
+    }
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        switch (readType) {
+          case FIXED_LENGTH_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case INT_LONG_BACKED_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case VARBINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case VARCHAR:
+            ((IcebergArrowVectors.VarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case FIXED_WIDTH_BINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+            break;
+          case BOOLEAN:
+            vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+            break;
+          case INT:
+            vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+            break;
+          case LONG:
+            vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+            break;
+          case FLOAT:
+            vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+            break;
+          case DOUBLE:
+            vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+            break;
+        }
+      }
+    }
+    Preconditions.checkState(vec.getValueCount() == numValsToRead,
+        "Number of values read, %s, does not equal expected, %s", vec.getValueCount(), numValsToRead);
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private void allocateFieldVector() {
+    if (allPagesDictEncoded) {
+      Field field = new Field(
+          icebergField.name(),
+          new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null),
+          null);
+      this.vec = field.createVector(rootAlloc);
+      ((IntVector) vec).allocateNew(batchSize);
+      typeWidth = IntVector.TYPE_WIDTH;
+    } else {
+      PrimitiveType primitive = columnDescriptor.getPrimitiveType();
+      if (primitive.getOriginalType() != null) {
+        switch (columnDescriptor.getPrimitiveType().getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+          case BSON:
+            this.vec = new IcebergArrowVectors.VarcharArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
+            vec.allocateNewSafe();
+            readType = ReadType.VARCHAR;
+            typeWidth = UNKNOWN_WIDTH;
+            break;
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((IntVector) vec).allocateNew(batchSize);
+            readType = ReadType.INT;
+            typeWidth = IntVector.TYPE_WIDTH;
+            break;
+          case DATE:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((DateDayVector) vec).allocateNew(batchSize);
+            readType = ReadType.INT;
+            typeWidth = IntVector.TYPE_WIDTH;
+            break;
+          case INT_64:
+          case TIMESTAMP_MILLIS:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((BigIntVector) vec).allocateNew(batchSize);
+            readType = ReadType.LONG;
+            typeWidth = BigIntVector.TYPE_WIDTH;
+            break;
+          case TIMESTAMP_MICROS:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((TimeStampMicroTZVector) vec).allocateNew(batchSize);
+            readType = ReadType.LONG;
+            typeWidth = BigIntVector.TYPE_WIDTH;
+            break;
+          case DECIMAL:
+            DecimalMetadata decimal = primitive.getDecimalMetadata();
+            this.vec = new IcebergArrowVectors.DecimalArrowVector(icebergField.name(), rootAlloc,
+                decimal.getPrecision(), decimal.getScale());
+            ((DecimalVector) vec).allocateNew(batchSize);
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                readType = ReadType.FIXED_LENGTH_DECIMAL;
+                typeWidth = primitive.getTypeLength();
+                break;
+              case INT64:
+                readType = ReadType.INT_LONG_BACKED_DECIMAL;
+                typeWidth = BigIntVector.TYPE_WIDTH;
+                break;
+              case INT32:
+                readType = ReadType.INT_LONG_BACKED_DECIMAL;
+                typeWidth = IntVector.TYPE_WIDTH;
+                break;
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+            break;
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      } else {
+        switch (primitive.getPrimitiveTypeName()) {
+          case FIXED_LEN_BYTE_ARRAY:
+            int len = ((Types.FixedType) icebergField.type()).length();
+            this.vec = new IcebergArrowVectors.VarBinaryArrowVector(icebergField.name(), rootAlloc);
+            int factor = (len + DEFAULT_RECORD_BYTE_COUNT - 1) / DEFAULT_RECORD_BYTE_COUNT;
+            vec.setInitialCapacity(batchSize * factor);
 
 Review comment:
   It isn't clear why this uses factor instead of `len` to determine the initial allocation size. I think it should be number of records per batch * bytes per value (len). That means we can also get rid of `DEFAULT_RECORD_BYTE_COUNT` because it isn't used anywhere else.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367656124
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.isFixedLengthDecimal = false;
+    this.isVarWidthType = false;
+    this.isFixedWidthBinary = false;
+    this.isBooleanType = false;
+    this.isPaddedDecimal = false;
+    this.isIntType = false;
+    this.isLongType = false;
+    this.isFloatType = false;
+    this.isDoubleType = false;
+    this.vectorizedColumnIterator = null;
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      typeWidth = allocateFieldVector();
+    }
+    vec.setValueCount(0);
+    nullabilityHolder = new NullabilityHolder(batchSize);
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        if (isFixedLengthDecimal) {
+          vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+          ((IcebergDecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+        } else if (isFixedWidthBinary) {
+          // Fixed width binary type values are stored in an IcebergVarBinaryArrowVector as well
+          if (vec instanceof IcebergVarBinaryArrowVector) {
+            ((IcebergVarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          }
+          vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+        } else if (isVarWidthType) {
+          if (vec instanceof IcebergVarcharArrowVector) {
+            ((IcebergVarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          } else if (vec instanceof IcebergVarBinaryArrowVector) {
+            ((IcebergVarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          }
+          vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+        } else if (isBooleanType) {
+          vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+        } else if (isPaddedDecimal) {
+          ((IcebergDecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+        } else if (isIntType) {
+          vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+        } else if (isLongType) {
+          vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+        } else if (isFloatType) {
+          vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+        } else if (isDoubleType) {
+          vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+        }
+      }
+    }
+    if (vec.getValueCount() != numValsToRead) {
+      throw new IllegalStateException("Number of values read into the vector, " +
+          vec.getValueCount() + " is not the same as the expected count of " + numValsToRead);
+    }
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private int allocateFieldVector() {
 
 Review comment:
   I'd normally expect an `allocateSomething` method to return the something, not a related int. Could this set the type width and return `void` instead? Or can the type width be derived from the vector that was allocated so this can return one thing?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368103333
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
 
 Review comment:
   On editors like Intellij, it tells me what the parameter name is. 
   Example:
   ```new ArrowType.Int(Integer.SIZE, isSigned: true);```
   
   Not sure I agree with having to add a comment for every parameter. Method signature should be enough, no?
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380869713
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,912 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.column.Dictionary;
+
+public final class VectorizedParquetValuesReader extends BaseVectorizedParquetValuesReader {
+
+  public VectorizedParquetValuesReader(int bitWidth, int maxDefLevel, boolean setArrowValidityVector) {
+    super(bitWidth, maxDefLevel, setArrowValidityVector);
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryIds(vector, idx, numValues, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(bufferIdx * typeWidth, valuesReader.readLong());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedLongs(vector,
+                idx, numValues, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(bufferIdx * typeWidth, valuesReader.readInteger());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedIntegers(vector, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(bufferIdx * typeWidth, valuesReader.readFloat());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFloats(vector, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(bufferIdx * typeWidth, valuesReader.readDouble());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(),  bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedDoubles(vector, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx, nullabilityHolder);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx, nullabilityHolder);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFixedWidthBinary(vector, typeWidth, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ByteBuffer buffer = dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).toByteBuffer();
+              vector.getDataBuffer().setBytes(idx * typeWidth, buffer.array(),
+                  buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              nullabilityHolder.setNotNull(bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              nullabilityHolder.setNotNull(bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
 
 Review comment:
   This is for the eagerDecode case. Ultimately, the dictionary based decoder is called - 
   `
   dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFixedLengthDecimals(vector, typeWidth, idx, num, dict, nullabilityHolder);
   `

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367669984
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedColumnIterator.java
 ##########
 @@ -0,0 +1,258 @@
+/*
+ * 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 org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * Vectorized version of the ColumnIterator that reads column values in data pages of a column in a row group in a
+ * batched fashion.
+ */
+public class VectorizedColumnIterator {
+
+  private final ColumnDescriptor desc;
+  private final VectorizedPageIterator vectorizedPageIterator;
+
+  // state reset for each row group
+  private PageReader columnPageReader = null;
+  private long totalValuesCount = 0L;
+  private long valuesRead = 0L;
+  private long advanceNextPageCount = 0L;
+  private final int batchSize;
+
+  public VectorizedColumnIterator(ColumnDescriptor desc, String writerVersion, int batchSize) {
+    this.desc = desc;
 
 Review comment:
   Only flat schemas are supported, right? Should this add some preconditions to validate that it isn't passed a column descriptor for a nested column?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368171781
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
 
 Review comment:
   This class doesn't have any code from Spark.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367632476
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
 
 Review comment:
   What is `true`? We usually add comments to clarify like this: `fs.delete(path, true /* delete recursively */)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r382850197
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,484 @@
+/*
+ * 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 org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.BasePageIterator;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+public class VectorizedPageIterator extends BasePageIterator {
+  private final boolean setArrowValidityVector;
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, boolean setValidityVector) {
+    super(desc, writerVersion);
+    this.setArrowValidityVector = setValidityVector;
+  }
+
+  private boolean eagerDecodeDictionary;
+  private ValuesAsBytesReader plainValuesReader = null;
+  private VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader = null;
+  private boolean allPagesDictEncoded;
+  private VectorizedParquetValuesReader vectorizedDefinitionLevelReader;
+
+  // Dictionary is set per row group
+  public void setDictionaryForColumn(Dictionary dict, boolean allDictEncoded) {
+    this.dictionary = dict;
+    this.allPagesDictEncoded = allDictEncoded;
+  }
+
+  @Override
+  protected void reset() {
 
 Review comment:
   Should this be in `BasePageIterator` and not here?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367656702
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
 
 Review comment:
   `ra` isn't very descriptive. Can we use `allocator` instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380819445
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java
 ##########
 @@ -0,0 +1,202 @@
+/*
+ * 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.parquet;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class BasePageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(BasePageIterator.class);
+
+  protected final ColumnDescriptor desc;
+  protected final String writerVersion;
+
+  // iterator state
+  protected boolean hasNext = false;
+  protected int triplesRead = 0;
+  protected int currentDL = 0;
+  protected int currentRL = 0;
+
+  // page bookkeeping
+  protected Dictionary dictionary = null;
+  protected DataPage page = null;
+  protected int triplesCount = 0;
+  protected Encoding valueEncoding = null;
+  protected IntIterator definitionLevels = null;
+  protected IntIterator repetitionLevels = null;
+  protected ValuesReader vectorizedDefinitionLevelReader = null;
+  protected ValuesReader values = null;
+
+  protected BasePageIterator(ColumnDescriptor descriptor, String writerVersion) {
+    this.desc = descriptor;
+    this.writerVersion = writerVersion;
+  }
+
+  protected abstract void reset();
+
+  protected abstract boolean supportsVectorizedReads();
+
+  protected abstract IntIterator newNonVectorizedDefinitionLevelReader(ValuesReader dlReader);
+
+  protected abstract ValuesReader newVectorizedDefinitionLevelReader(ColumnDescriptor descriptor);
+
+  protected abstract void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount);
+
+  public void setPage(DataPage page) {
+    Preconditions.checkNotNull(page, "Cannot read from null page");
+    this.page = page;
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    this.hasNext = triplesRead < triplesCount;
+  }
+
+  protected void initFromPage(DataPageV1 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    ValuesReader dlReader = null;
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      dlReader = initPage.getDlEncoding().getValuesReader(desc, ValuesType.DEFINITION_LEVEL);
+      this.definitionLevels = newNonVectorizedDefinitionLevelReader(dlReader);
+    }
+    ValuesReader rlReader = initPage.getRlEncoding().getValuesReader(desc, ValuesType.REPETITION_LEVEL);
+    this.repetitionLevels = new PageIterator.ValuesReaderIntIterator(rlReader);
+    try {
+      BytesInput bytes = initPage.getBytes();
+      LOG.debug("page size {} bytes and {} records", bytes.size(), triplesCount);
+      LOG.debug("reading repetition levels at 0");
+      ByteBufferInputStream in = bytes.toInputStream();
+      rlReader.initFromPage(triplesCount, in);
+      LOG.debug("reading definition levels at {}", in.position());
+      if (supportsVectorizedReads()) {
+        this.vectorizedDefinitionLevelReader.initFromPage(triplesCount, in);
+      } else {
+        dlReader.initFromPage(triplesCount, in);
+      }
+      LOG.debug("reading data at {}", in.position());
+      initDataReader(initPage.getValueEncoding(), in, initPage.getValueCount());
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  protected void initFromPage(DataPageV2 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    this.repetitionLevels = newRLEIterator(desc.getMaxRepetitionLevel(), initPage.getRepetitionLevels());
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      this.definitionLevels = newRLEIterator(desc.getMaxDefinitionLevel(), initPage.getDefinitionLevels());
+    }
+    LOG.debug("page data size {} bytes and {} records", initPage.getData().size(), triplesCount);
+    try {
+      initDataReader(initPage.getDataEncoding(), initPage.getData().toInputStream(), triplesCount);
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
+    try {
+      if (maxLevel == 0) {
+        return new NullIntIterator();
+      }
+      return new RLEIntIterator(
+          new RunLengthBitPackingHybridDecoder(
+              BytesUtils.getWidthFromMaxInt(maxLevel),
+              bytes.toInputStream()));
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read levels in page for col " + desc, e);
+    }
+  }
+
+  public void setDictionary(Dictionary dict) {
+    this.dictionary = dict;
+  }
+
+  protected abstract static class IntIterator {
+    abstract int nextInt();
+  }
+
+  static class ValuesReaderIntIterator extends IntIterator {
 
 Review comment:
   This would work with a vectorized `ValuesReader` right?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367726571
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
 
 Review comment:
   Why doesn't this check `currentValue`?
   
   It looks like this is using the vectorized reader as a definition-level reader that will read runs of data values. Is that not the case?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367638389
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
+        break;
+      case DATE:
+        arrowType = new ArrowType.Date(DateUnit.DAY);
+        break;
+      case STRUCT:
+        final StructType struct = field.type().asStructType();
+        arrowType = ArrowType.Struct.INSTANCE;
+
+        for (NestedField nested : struct.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case LIST:
+        final ListType listType = field.type().asListType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        for (NestedField nested : listType.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case MAP:
+        //Maps are represented as List<Struct<key, value>>
 
 Review comment:
   There's now a `Map` type in metadata: https://github.com/apache/arrow/blob/master/format/Schema.fbs#L84-L87
   
   Should we use that?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380398313
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
 
 Review comment:
   Invalid Javadoc comment start?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367640160
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
+        break;
+      case DATE:
+        arrowType = new ArrowType.Date(DateUnit.DAY);
+        break;
+      case STRUCT:
+        final StructType struct = field.type().asStructType();
+        arrowType = ArrowType.Struct.INSTANCE;
+
+        for (NestedField nested : struct.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case LIST:
+        final ListType listType = field.type().asListType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        for (NestedField nested : listType.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case MAP:
+        //Maps are represented as List<Struct<key, value>>
+        metadata = ImmutableMap.of(ORIGINAL_TYPE, MAP_TYPE);
+        final MapType mapType = field.type().asMapType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        final List<Field> entryFields = Lists.newArrayList(
+            convert(required(0, MAP_KEY, mapType.keyType())),
+            convert(optional(0, MAP_VALUE, mapType.valueType()))
 
 Review comment:
   This can use the fields from the Iceberg map type:
   
   ```
     List<Field> entryFields = Lists.transform(mapType.fields(), ArrowSchemaUtil::convert);
   ```
   
   Using the underlying fields will also pass whether the value is required or optional correctly. You can also access that using `MapType.isValueOptional()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380822203
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java
 ##########
 @@ -0,0 +1,202 @@
+/*
+ * 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.parquet;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class BasePageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(BasePageIterator.class);
+
+  protected final ColumnDescriptor desc;
+  protected final String writerVersion;
+
+  // iterator state
+  protected boolean hasNext = false;
+  protected int triplesRead = 0;
+  protected int currentDL = 0;
+  protected int currentRL = 0;
+
+  // page bookkeeping
+  protected Dictionary dictionary = null;
+  protected DataPage page = null;
+  protected int triplesCount = 0;
+  protected Encoding valueEncoding = null;
+  protected IntIterator definitionLevels = null;
+  protected IntIterator repetitionLevels = null;
+  protected ValuesReader vectorizedDefinitionLevelReader = null;
+  protected ValuesReader values = null;
+
+  protected BasePageIterator(ColumnDescriptor descriptor, String writerVersion) {
+    this.desc = descriptor;
+    this.writerVersion = writerVersion;
+  }
+
+  protected abstract void reset();
+
+  protected abstract boolean supportsVectorizedReads();
+
+  protected abstract IntIterator newNonVectorizedDefinitionLevelReader(ValuesReader dlReader);
+
+  protected abstract ValuesReader newVectorizedDefinitionLevelReader(ColumnDescriptor descriptor);
+
+  protected abstract void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount);
+
+  public void setPage(DataPage page) {
+    Preconditions.checkNotNull(page, "Cannot read from null page");
+    this.page = page;
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    this.hasNext = triplesRead < triplesCount;
+  }
+
+  protected void initFromPage(DataPageV1 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    ValuesReader dlReader = null;
+    if (supportsVectorizedReads()) {
 
 Review comment:
   Another way to clean up this and its subclasses is to use a method like the one for value reader initialization, `initDataReader`. Couldn't this call `initDefinitionLevelReader` with `triplesCount`, `in`, and `desc`? Then the subclasses would be responsible for building and initializing their own DL reader.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r382848663
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,247 @@
+/*
+ * 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 org.apache.parquet.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.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the VectorizedRleValuesReader class in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
 
 Review comment:
   Nit: Using `tr` tags seems odd here, outside of a table.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367667500
 
 

 ##########
 File path: versions.lock
 ##########
 @@ -95,9 +96,9 @@ org.antlr:antlr4-runtime:4.7 (1 constraints: 7a0e125f)
 org.antlr:stringtemplate:3.2.1 (1 constraints: c10a3bc6)
 org.apache.ant:ant:1.9.1 (3 constraints: a721ed14)
 org.apache.ant:ant-launcher:1.9.1 (1 constraints: 69082485)
-org.apache.arrow:arrow-format:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-memory:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-vector:0.10.0 (1 constraints: e90c9734)
+org.apache.arrow:arrow-format:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-memory:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-vector:0.14.1 (2 constraints: 2012a545)
 
 Review comment:
   0.15.1 is out. Does that version work?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367666190
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.isFixedLengthDecimal = false;
+    this.isVarWidthType = false;
+    this.isFixedWidthBinary = false;
+    this.isBooleanType = false;
+    this.isPaddedDecimal = false;
+    this.isIntType = false;
+    this.isLongType = false;
+    this.isFloatType = false;
+    this.isDoubleType = false;
+    this.vectorizedColumnIterator = null;
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      typeWidth = allocateFieldVector();
+    }
+    vec.setValueCount(0);
+    nullabilityHolder = new NullabilityHolder(batchSize);
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        if (isFixedLengthDecimal) {
+          vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+          ((IcebergDecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+        } else if (isFixedWidthBinary) {
+          // Fixed width binary type values are stored in an IcebergVarBinaryArrowVector as well
+          if (vec instanceof IcebergVarBinaryArrowVector) {
+            ((IcebergVarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          }
+          vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+        } else if (isVarWidthType) {
+          if (vec instanceof IcebergVarcharArrowVector) {
+            ((IcebergVarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          } else if (vec instanceof IcebergVarBinaryArrowVector) {
+            ((IcebergVarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          }
+          vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+        } else if (isBooleanType) {
+          vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+        } else if (isPaddedDecimal) {
+          ((IcebergDecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+        } else if (isIntType) {
+          vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+        } else if (isLongType) {
+          vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+        } else if (isFloatType) {
+          vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+        } else if (isDoubleType) {
+          vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+        }
+      }
+    }
+    if (vec.getValueCount() != numValsToRead) {
+      throw new IllegalStateException("Number of values read into the vector, " +
+          vec.getValueCount() + " is not the same as the expected count of " + numValsToRead);
+    }
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private int allocateFieldVector() {
+    if (allPagesDictEncoded) {
+      Field field = new Field(
+          icebergField.name(),
+          new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null),
+          null);
+      this.vec = field.createVector(rootAlloc);
+      ((IntVector) vec).allocateNew(batchSize);
+      return IntVector.TYPE_WIDTH;
+    } else {
+      PrimitiveType primitive = columnDescriptor.getPrimitiveType();
+      if (primitive.getOriginalType() != null) {
+        switch (columnDescriptor.getPrimitiveType().getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+          case BSON:
+            this.vec = new IcebergVarcharArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
+            vec.allocateNewSafe();
+            return UNKNOWN_WIDTH;
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((IntVector) vec).allocateNew(batchSize);
+            return IntVector.TYPE_WIDTH;
+          case DATE:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((DateDayVector) vec).allocateNew(batchSize);
+            return IntVector.TYPE_WIDTH;
+          case INT_64:
+          case TIMESTAMP_MILLIS:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((BigIntVector) vec).allocateNew(batchSize);
+            return BigIntVector.TYPE_WIDTH;
+          case TIMESTAMP_MICROS:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((TimeStampMicroTZVector) vec).allocateNew(batchSize);
+            return BigIntVector.TYPE_WIDTH;
+          case DECIMAL:
+            DecimalMetadata decimal = primitive.getDecimalMetadata();
+            this.vec = new IcebergDecimalArrowVector(icebergField.name(), rootAlloc, decimal.getPrecision(),
+                decimal.getScale());
+            ((DecimalVector) vec).allocateNew(batchSize);
+            switch (primitive.getPrimitiveTypeName()) {
+              case BINARY:
+              case FIXED_LEN_BYTE_ARRAY:
+                return primitive.getTypeLength();
+              case INT64:
+                return BigIntVector.TYPE_WIDTH;
+              case INT32:
+                return IntVector.TYPE_WIDTH;
+              default:
+                throw new UnsupportedOperationException(
+                    "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+            }
+          default:
+            throw new UnsupportedOperationException(
+                "Unsupported logical type: " + primitive.getOriginalType());
+        }
+      } else {
+        switch (primitive.getPrimitiveTypeName()) {
+          case FIXED_LEN_BYTE_ARRAY:
+            int len = ((Types.FixedType) icebergField.type()).length();
+            this.vec = new IcebergVarBinaryArrowVector(icebergField.name(), rootAlloc);
+            int factor = (len + DEFAULT_RECORD_BYTE_COUNT - 1) / DEFAULT_RECORD_BYTE_COUNT;
+            vec.setInitialCapacity(batchSize * factor);
+            vec.allocateNew();
+            return len;
+          case BINARY:
+            this.vec = new IcebergVarBinaryArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
+            vec.allocateNewSafe();
+            return UNKNOWN_WIDTH;
+          case INT32:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((IntVector) vec).allocateNew(batchSize);
+            return IntVector.TYPE_WIDTH;
+          case FLOAT:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((Float4Vector) vec).allocateNew(batchSize);
+            return Float4Vector.TYPE_WIDTH;
+          case BOOLEAN:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((BitVector) vec).allocateNew(batchSize);
+            return UNKNOWN_WIDTH;
+          case INT64:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((BigIntVector) vec).allocateNew(batchSize);
+            return BigIntVector.TYPE_WIDTH;
+          case DOUBLE:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((Float8Vector) vec).allocateNew(batchSize);
+            return Float8Vector.TYPE_WIDTH;
+          default:
+            throw new UnsupportedOperationException("Unsupported type: " + primitive);
+        }
+      }
+    }
+  }
+
+  @Override
+  public void setRowGroupInfo(PageReadStore source, Map<ColumnPath, ColumnChunkMetaData> metadata) {
+    ColumnChunkMetaData chunkMetaData = metadata.get(ColumnPath.get(columnDescriptor.getPath()));
+    allPagesDictEncoded = !ParquetUtil.hasNonDictionaryPages(chunkMetaData);
+    dictionary = vectorizedColumnIterator.setRowGroupInfo(source, allPagesDictEncoded);
+  }
+
+  @Override
+  public void reuseContainers(boolean reuse) {
+    this.reuseContainers = reuse;
+  }
+
+  @Override
+  public String toString() {
+    return columnDescriptor.toString();
+  }
+
+  public static final VectorizedArrowReader NULL_VALUES_READER =
+      new VectorizedArrowReader() {
+        @Override
+        public VectorHolder read(int numValsToRead) {
+          return VectorHolder.NULL_VECTOR_HOLDER;
+        }
+
+        @Override
+        public void setRowGroupInfo(PageReadStore source, Map<ColumnPath, ColumnChunkMetaData> metadata) {
+        }
+      };
+
+  private static boolean isFixedLengthDecimal(ColumnDescriptor desc) {
+    PrimitiveType primitive = desc.getPrimitiveType();
+    return primitive.getOriginalType() != null &&
+        primitive.getOriginalType() == OriginalType.DECIMAL &&
+        (primitive.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY ||
+            primitive.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.BINARY);
+  }
+
+  private static boolean isIntLongBackedDecimal(ColumnDescriptor desc) {
+    PrimitiveType primitive = desc.getPrimitiveType();
+    return primitive.getOriginalType() != null &&
+        primitive.getOriginalType() == OriginalType.DECIMAL &&
+        (primitive.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT64 ||
+            primitive.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT32);
+  }
+
+  private static boolean isVarWidthType(ColumnDescriptor desc) {
+    PrimitiveType primitive = desc.getPrimitiveType();
+    OriginalType originalType = primitive.getOriginalType();
+    if (originalType != null &&
+        originalType != OriginalType.DECIMAL &&
+        (originalType == OriginalType.ENUM ||
+            originalType == OriginalType.JSON ||
+            originalType == OriginalType.UTF8 ||
+            originalType == OriginalType.BSON)) {
+      return true;
+    }
+    if (originalType == null && primitive.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.BINARY) {
+      return true;
+    }
+    return false;
+  }
+
+  private static boolean isBooleanType(ColumnDescriptor desc) {
+    PrimitiveType primitive = desc.getPrimitiveType();
+    OriginalType originalType = primitive.getOriginalType();
+    return originalType == null && primitive.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.BOOLEAN;
+  }
+
+  private static boolean isFixedWidthBinary(ColumnDescriptor desc) {
+    PrimitiveType primitive = desc.getPrimitiveType();
+    OriginalType originalType = primitive.getOriginalType();
+    if (originalType == null &&
+        primitive.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) {
+      return true;
+    }
+    return false;
+  }
+
+  private static boolean isIntType(ColumnDescriptor desc) {
+    return desc.getPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT32;
+  }
+
+  private static boolean isLongType(ColumnDescriptor desc) {
+    return desc.getPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.INT64;
+  }
+
+  private static boolean isDoubleType(ColumnDescriptor desc) {
+    return desc.getPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.DOUBLE;
+  }
+
+  private static boolean isFloatType(ColumnDescriptor desc) {
+    return desc.getPrimitiveType().getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FLOAT;
+  }
+
+  private static Map<ColumnPath, Boolean> buildColumnDictEncodedMap(BlockMetaData blockMetaData) {
 
 Review comment:
   Doesn't look like this is used anywhere?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367656409
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.isFixedLengthDecimal = false;
+    this.isVarWidthType = false;
+    this.isFixedWidthBinary = false;
+    this.isBooleanType = false;
+    this.isPaddedDecimal = false;
+    this.isIntType = false;
+    this.isLongType = false;
+    this.isFloatType = false;
+    this.isDoubleType = false;
+    this.vectorizedColumnIterator = null;
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      typeWidth = allocateFieldVector();
+    }
+    vec.setValueCount(0);
+    nullabilityHolder = new NullabilityHolder(batchSize);
 
 Review comment:
   Why is a new nullability holder always used instead of reusing the existing ones?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380403699
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java
 ##########
 @@ -0,0 +1,408 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.Dictionary;
+
+public class VectorizedDictionaryEncodedParquetValuesReader extends BaseVectorizedParquetValuesReader {
+
+  public VectorizedDictionaryEncodedParquetValuesReader(int maxDefLevel, boolean setValidityVector) {
+    super(maxDefLevel, setValidityVector);
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  void readBatchOfDictionaryIds(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead,
 
 Review comment:
   Style: we don't typically use `final` for method arguments, or newlines between each one. We do treat method arguments as final, though.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368177847
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorHolder.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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;
+
+import javax.annotation.Nullable;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+
+/**
+ * Container class for holding the Arrow vector holding a batch of values along with other state needed for reading
+ * values out of it.
+ */
+public class VectorHolder {
 
 Review comment:
   The idea here was to provide enough state for the consumers to efficiently read the values in arrow vectors. 
   For consumers like Spark, which is not shown in this PR, we have 
   https://github.com/samarthjain/incubator-iceberg/blob/vectorized-reusecontainers/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java#L587
   
   for reading dictionary encoded values
   
   and 
   https://github.com/samarthjain/incubator-iceberg/blob/vectorized-reusecontainers/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java#L564
   
   for reading non dictionary encoded values.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367665496
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.isFixedLengthDecimal = false;
+    this.isVarWidthType = false;
+    this.isFixedWidthBinary = false;
+    this.isBooleanType = false;
+    this.isPaddedDecimal = false;
+    this.isIntType = false;
+    this.isLongType = false;
+    this.isFloatType = false;
+    this.isDoubleType = false;
+    this.vectorizedColumnIterator = null;
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      typeWidth = allocateFieldVector();
+    }
+    vec.setValueCount(0);
+    nullabilityHolder = new NullabilityHolder(batchSize);
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        if (isFixedLengthDecimal) {
+          vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+          ((IcebergDecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+        } else if (isFixedWidthBinary) {
+          // Fixed width binary type values are stored in an IcebergVarBinaryArrowVector as well
+          if (vec instanceof IcebergVarBinaryArrowVector) {
+            ((IcebergVarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          }
+          vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+        } else if (isVarWidthType) {
+          if (vec instanceof IcebergVarcharArrowVector) {
+            ((IcebergVarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          } else if (vec instanceof IcebergVarBinaryArrowVector) {
+            ((IcebergVarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          }
+          vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+        } else if (isBooleanType) {
+          vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+        } else if (isPaddedDecimal) {
+          ((IcebergDecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+          vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+        } else if (isIntType) {
+          vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+        } else if (isLongType) {
+          vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+        } else if (isFloatType) {
+          vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+        } else if (isDoubleType) {
+          vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+        }
+      }
+    }
+    if (vec.getValueCount() != numValsToRead) {
+      throw new IllegalStateException("Number of values read into the vector, " +
 
 Review comment:
   Using `Preconditions` could save some space and make this a format string:
   
   ```java
   Preconditions.checkState(vec.getValueCount() == numValsToRead,
       "Number of values read, %s, does not equal expected, %s", vec.getValueCount(), numValsToRead);
   ```
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367729572
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
 
 Review comment:
   I think that `getBytesUnsafe` will result in a copy, although it will be reused if the same dictionary value is seen again. Is it possible to get a `ByteBuffer` instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380824873
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,912 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.column.Dictionary;
+
+public final class VectorizedParquetValuesReader extends BaseVectorizedParquetValuesReader {
+
+  public VectorizedParquetValuesReader(int bitWidth, int maxDefLevel, boolean setArrowValidityVector) {
+    super(bitWidth, maxDefLevel, setArrowValidityVector);
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryIds(vector, idx, numValues, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(bufferIdx * typeWidth, valuesReader.readLong());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedLongs(vector,
+                idx, numValues, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(bufferIdx * typeWidth, valuesReader.readInteger());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedIntegers(vector, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(bufferIdx * typeWidth, valuesReader.readFloat());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFloats(vector, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(bufferIdx * typeWidth, valuesReader.readDouble());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(),  bufferIdx);
+              } else {
+                nullabilityHolder.setNotNull(bufferIdx);
+              }
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedDoubles(vector, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(dictionaryEncodedValuesReader.readInteger()));
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx, nullabilityHolder);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx, nullabilityHolder);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedFixedWidthBinary(vector, typeWidth, idx,
+                num, dict, nullabilityHolder);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ByteBuffer buffer = dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).toByteBuffer();
+              vector.getDataBuffer().setBytes(idx * typeWidth, buffer.array(),
+                  buffer.position() + buffer.arrayOffset(), buffer.limit() - buffer.position());
+              if (setArrowValidityVector) {
+                BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+              } else {
+                nullabilityHolder.setNotNull(idx);
+              }
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              nullabilityHolder.setNotNull(bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              nullabilityHolder.setNotNull(bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
 
 Review comment:
   Why does this class have both dictionary and non-dictionary methods?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387370583
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,471 @@
+/*
+ * 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 org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.BasePageIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+public class VectorizedPageIterator extends BasePageIterator {
+  private final boolean setArrowValidityVector;
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, boolean setValidityVector) {
+    super(desc, writerVersion);
+    this.setArrowValidityVector = setValidityVector;
+  }
+
+  private boolean eagerDecodeDictionary;
+  private ValuesAsBytesReader plainValuesReader = null;
+  private VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader = null;
+  private boolean allPagesDictEncoded;
+  private VectorizedParquetDefinitionLevelReader vectorizedDefinitionLevelReader;
+
+  public void setAllPagesDictEncoded(boolean allDictEncoded) {
+    this.allPagesDictEncoded = allDictEncoded;
+  }
+
+  @Override
+  protected void reset() {
+    super.reset();
+    this.plainValuesReader = null;
+    this.vectorizedDefinitionLevelReader = null;
+  }
+
+  /**
+   * Method for reading a batch of dictionary ids from the dicitonary encoded data pages. Like definition levels,
+   * dictionary ids in Parquet are RLE/bin-packed encoded as well.
+   */
+  public int nextBatchDictionaryIds(
+      final IntVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    vectorizedDefinitionLevelReader.readBatchOfDictionaryIds(
+        vector,
+        numValsInVector,
+        actualBatchSize,
+        holder,
+        dictionaryEncodedValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT32 data type
+   */
+  public int nextBatchIntegers(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT64 data type
+   */
+  public int nextBatchLongs(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of TIMESTAMP_MILLIS data type. In iceberg, TIMESTAMP
+   * is always represented in micro-seconds. So we multiply values stored in millis with 1000
+   * before writing them to the vector.
+   */
+  public int nextBatchTimestampMillis(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedTimestampMillis(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfTimestampMillis(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of FLOAT data type.
+   */
+  public int nextBatchFloats(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of DOUBLE data type
+   */
+  public int nextBatchDoubles(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  private int getActualBatchSize(int expectedBatchSize) {
+    return Math.min(expectedBatchSize, triplesCount - triplesRead);
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by INT32 and INT64 parquet data types. Since Arrow stores all
+   * decimals in 16 bytes, byte arrays are appropriately padded before being written to Arrow data buffers.
+   */
+  public int nextBatchIntLongBackedDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader
+          .readBatchOfDictionaryEncodedIntLongBackedDecimals(
+              vector,
+              numValsInVector,
+              typeWidth,
+              actualBatchSize,
+              nullabilityHolder,
+              dictionaryEncodedValuesReader,
+              dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfIntLongBackedDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by fixed length byte array parquet data type. Arrow stores all
+   * decimals in 16 bytes. This method provides the necessary padding to the decimals read. Moreover, Arrow interprets
+   * the decimals in Arrow buffer as little endian. Parquet stores fixed length decimals as big endian. So, this method
+   * uses {@link DecimalVector#setBigEndian(int, byte[])} method so that the data in Arrow vector is indeed little
+   * endian.
+   */
+  public int nextBatchFixedLengthDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of variable width data type (ENUM, JSON, UTF8, BSON).
+   */
+  public int nextBatchVarWidthType(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of fixed width binary type (e.g. BYTE[7]). Spark does not support fixed width binary
+   * data type. To work around this limitation, the data is read as fixed width binary from parquet and stored in a
+   * {@link VarBinaryVector} in Arrow.
+   */
+  public int nextBatchFixedWidthBinary(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of booleans.
+   */
+  public int nextBatchBoolean(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    vectorizedDefinitionLevelReader
+        .readBatchOfBooleans(vector, numValsInVector, actualBatchSize,
+            nullabilityHolder, plainValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  @Override
+  protected void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) {
+    ValuesReader previousReader = plainValuesReader;
+    this.eagerDecodeDictionary = dataEncoding.usesDictionary() && dictionary != null &&
+        (ParquetUtil.isIntType(desc.getPrimitiveType()) || !allPagesDictEncoded);
 
 Review comment:
   @samarthjain, in a follow-up, let's update the logic here to something other than just eagerly decoding integers. We probably want to do this for 4-byte floats, for example.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380406739
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedDictionaryEncodedParquetValuesReader.java
 ##########
 @@ -0,0 +1,408 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.parquet.column.Dictionary;
+
+public class VectorizedDictionaryEncodedParquetValuesReader extends BaseVectorizedParquetValuesReader {
+
+  public VectorizedDictionaryEncodedParquetValuesReader(int maxDefLevel, boolean setValidityVector) {
+    super(maxDefLevel, setValidityVector);
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  void readBatchOfDictionaryIds(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead,
+      NullabilityHolder nullabilityHolder) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            nullabilityHolder.setNotNull(idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            nullabilityHolder.setNotNull(idx);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  void readBatchOfDictionaryEncodedLongs(
+      FieldVector vector,
+      int index,
+      int numValuesToRead,
+      Dictionary dict,
+      NullabilityHolder nullabilityHolder) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            if (setArrowValidityVector) {
 
 Review comment:
   Does this `if` affect performance? Seems like we could get this done by using a different method, like `readBatchOfDictionaryEncodedLongs(vec, idx, size, dict, nullabilityHolder)` vs `readBatchOfDictionaryEncodedLongs(vec, idx, size, dict)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380867421
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,444 @@
+/*
+ * 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 org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.BasePageIterator;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+public class VectorizedPageIterator extends BasePageIterator {
+  private final boolean setArrowValidityVector;
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, boolean setValidityVector) {
+    super(desc, writerVersion);
+    this.setArrowValidityVector = setValidityVector;
+  }
+
+  private boolean eagerDecodeDictionary;
+  private ValuesAsBytesReader plainValuesReader = null;
+  private VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader = null;
+  private boolean allPagesDictEncoded;
+
+  // Dictionary is set per row group
+  public void setDictionaryForColumn(Dictionary dict, boolean allDictEncoded) {
+    this.dictionary = dict;
+    this.allPagesDictEncoded = allDictEncoded;
+  }
+
+  @Override
+  protected void reset() {
+    this.page = null;
+    this.triplesCount = 0;
+    this.triplesRead = 0;
+    this.repetitionLevels = null;
+    this.plainValuesReader = null;
+    this.vectorizedDefinitionLevelReader = null;
+    this.hasNext = false;
+  }
+
+  public int currentPageCount() {
+    return triplesCount;
+  }
+
+  public boolean hasNext() {
+    return hasNext;
+  }
+
+  /**
+   * Method for reading a batch of dictionary ids from the dicitonary encoded data pages. Like definition levels,
+   * dictionary ids in Parquet are RLE/bin-packed encoded as well.
+   */
+  public int nextBatchDictionaryIds(
+      final IntVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    ((VectorizedParquetValuesReader) vectorizedDefinitionLevelReader).readBatchOfDictionaryIds(
 
 Review comment:
   There is a dependency issue here. VectorizedParquetValuesReader is in arrow module where as the base class BasePageIterator is in parquet module. The arrow module depends on parquet. So I cannot declare the type of vectorizedDefinitionLevelReader in BasePageIterator to VectorizedParquetValuesReader as it would result in a circular dependency.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387185834
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedPageIterator.java
 ##########
 @@ -0,0 +1,469 @@
+/*
+ * 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 org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.BasePageIterator;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.CorruptDeltaByteArrays;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.RequiresPreviousReader;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.io.ParquetDecodingException;
+
+public class VectorizedPageIterator extends BasePageIterator {
+  private final boolean setArrowValidityVector;
+
+  public VectorizedPageIterator(ColumnDescriptor desc, String writerVersion, boolean setValidityVector) {
+    super(desc, writerVersion);
+    this.setArrowValidityVector = setValidityVector;
+  }
+
+  private boolean eagerDecodeDictionary;
+  private ValuesAsBytesReader plainValuesReader = null;
+  private VectorizedDictionaryEncodedParquetValuesReader dictionaryEncodedValuesReader = null;
+  private boolean allPagesDictEncoded;
+  private VectorizedParquetValuesReader vectorizedDefinitionLevelReader;
+
+  public void setAllPagesDictEncoded(boolean allDictEncoded) {
+    this.allPagesDictEncoded = allDictEncoded;
+  }
+
+  @Override
+  protected void reset() {
+    super.reset();
+    this.plainValuesReader = null;
+    this.vectorizedDefinitionLevelReader = null;
+  }
+
+  /**
+   * Method for reading a batch of dictionary ids from the dicitonary encoded data pages. Like definition levels,
+   * dictionary ids in Parquet are RLE/bin-packed encoded as well.
+   */
+  public int nextBatchDictionaryIds(
+      final IntVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    vectorizedDefinitionLevelReader.readBatchOfDictionaryIds(
+        vector,
+        numValsInVector,
+        actualBatchSize,
+        holder,
+        dictionaryEncodedValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT32 data type
+   */
+  public int nextBatchIntegers(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfIntegers(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of INT64 data type
+   */
+  public int nextBatchLongs(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfLongs(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of TIMESTAMP_MILLIS data type. In iceberg, TIMESTAMP
+   * is always represented in micro-seconds. So we multiply values stored in millis with 1000
+   * before writing them to the vector.
+   */
+  public int nextBatchTimestampMillis(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedTimestampMillis(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfTimestampMillis(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of FLOAT data type.
+   */
+  public int nextBatchFloats(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfFloats(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of values of DOUBLE data type
+   */
+  public int nextBatchDoubles(
+      final FieldVector vector, final int expectedBatchSize,
+      final int numValsInVector,
+      final int typeWidth, NullabilityHolder holder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfDoubles(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          holder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  private int getActualBatchSize(int expectedBatchSize) {
+    return Math.min(expectedBatchSize, triplesCount - triplesRead);
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by INT32 and INT64 parquet data types. Since Arrow stores all
+   * decimals in 16 bytes, byte arrays are appropriately padded before being written to Arrow data buffers.
+   */
+  public int nextBatchIntLongBackedDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader
+          .readBatchOfDictionaryEncodedIntLongBackedDecimals(
+              vector,
+              numValsInVector,
+              typeWidth,
+              actualBatchSize,
+              nullabilityHolder,
+              dictionaryEncodedValuesReader,
+              dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfIntLongBackedDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of decimals backed by fixed length byte array parquet data type. Arrow stores all
+   * decimals in 16 bytes. This method provides the necessary padding to the decimals read. Moreover, Arrow interprets
+   * the decimals in Arrow buffer as little endian. Parquet stores fixed length decimals as big endian. So, this method
+   * uses {@link DecimalVector#setBigEndian(int, byte[])} method so that the data in Arrow vector is indeed little
+   * endian.
+   */
+  public int nextBatchFixedLengthDecimal(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfFixedLengthDecimals(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading a batch of variable width data type (ENUM, JSON, UTF8, BSON).
+   */
+  public int nextBatchVarWidthType(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchVarWidth(
+          vector,
+          numValsInVector,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of fixed width binary type (e.g. BYTE[7]). Spark does not support fixed width binary
+   * data type. To work around this limitation, the data is read as fixed width binary from parquet and stored in a
+   * {@link VarBinaryVector} in Arrow.
+   */
+  public int nextBatchFixedWidthBinary(
+      final FieldVector vector, final int expectedBatchSize, final int numValsInVector,
+      final int typeWidth, NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    if (eagerDecodeDictionary) {
+      vectorizedDefinitionLevelReader.readBatchOfDictionaryEncodedFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          dictionaryEncodedValuesReader,
+          dictionary);
+    } else {
+      vectorizedDefinitionLevelReader.readBatchOfFixedWidthBinary(
+          vector,
+          numValsInVector,
+          typeWidth,
+          actualBatchSize,
+          nullabilityHolder,
+          plainValuesReader);
+    }
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  /**
+   * Method for reading batches of booleans.
+   */
+  public int nextBatchBoolean(
+      final FieldVector vector,
+      final int expectedBatchSize,
+      final int numValsInVector,
+      NullabilityHolder nullabilityHolder) {
+    final int actualBatchSize = getActualBatchSize(expectedBatchSize);
+    if (actualBatchSize <= 0) {
+      return 0;
+    }
+    vectorizedDefinitionLevelReader
+        .readBatchOfBooleans(vector, numValsInVector, actualBatchSize,
+            nullabilityHolder, plainValuesReader);
+    triplesRead += actualBatchSize;
+    this.hasNext = triplesRead < triplesCount;
+    return actualBatchSize;
+  }
+
+  @Override
+  protected void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) {
+    ValuesReader previousReader = plainValuesReader;
+    this.eagerDecodeDictionary = dataEncoding.usesDictionary() && dictionary != null && !allPagesDictEncoded;
 
 Review comment:
   Looks like this will only use eager decoding when not all pages are dictionary encoded.
   
   What about cases where there is no need to keep the dictionary around, like int columns?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] shawnding commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
shawnding commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r383646541
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/ValuesAsBytesReader.java
 ##########
 @@ -76,12 +91,29 @@ public final boolean readBoolean() {
     return value;
   }
 
+  /**
+   *
+   * @return 1 if true, 0 otherwise
+   */
+  public final int readBooleanAsInt() {
+    if (bitOffset == 0) {
+      currentByte = getByte();
+    }
+    int value = (currentByte & (1 << bitOffset)) >> bitOffset;
+    bitOffset += 1;
+    if (bitOffset == 8) {
+      bitOffset = 0;
+    }
+    return value;
+  }
+
   private byte getByte() {
     try {
       return (byte) valuesInputStream.read();
     } catch (IOException e) {
       throw new ParquetDecodingException("Failed to read a byte", e);
     }
   }
+
 
 Review comment:
   maybe unnecessary empty line.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380385226
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/NullabilityHolder.java
 ##########
 @@ -0,0 +1,58 @@
+/*
+ * 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;
+
+public class NullabilityHolder {
+  private final byte[] isNull;
+  private int numNulls;
+
+  public NullabilityHolder(int size) {
+    this.isNull = new byte[size];
+  }
+
+  public void setNull(int idx) {
+    isNull[idx] = 1;
+    numNulls++;
 
 Review comment:
   It doesn't look like `numNulls` is correct, since it is incremented here as well as in `setNotNull`. Even if `setNotNull` didn't increment it, the value would also be incorrect if the same position were set twice. Do we need to track the number of nulls? It doesn't look like this is called anywhere in this set of changes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r380401765
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,334 @@
+/*
+ * 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;
+
+import com.google.common.base.Preconditions;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private ReadType readType;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize,
+      boolean setArrowValidityVector) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.vectorizedColumnIterator = new VectorizedColumnIterator(desc, "", batchSize, setArrowValidityVector);
+  }
+
+  private VectorizedArrowReader() {
+    this.icebergField = null;
+    this.batchSize = DEFAULT_BATCH_SIZE;
+    this.columnDescriptor = null;
+    this.rootAlloc = null;
+    this.vectorizedColumnIterator = null;
+  }
+
+  private enum ReadType {
+    FIXED_LENGTH_DECIMAL, INT_LONG_BACKED_DECIMAL, VARCHAR, VARBINARY, FIXED_WIDTH_BINARY,
+    BOOLEAN, INT, LONG, FLOAT, DOUBLE
+  }
+
+  @Override
+  public VectorHolder read(int numValsToRead) {
+    if (vec == null || !reuseContainers) {
+      allocateFieldVector();
+      nullabilityHolder = new NullabilityHolder(batchSize);
+    } else {
+      vec.setValueCount(0);
+      nullabilityHolder.reset();
+    }
+    if (vectorizedColumnIterator.hasNext()) {
+      if (allPagesDictEncoded) {
+        vectorizedColumnIterator.nextBatchDictionaryIds((IntVector) vec, nullabilityHolder);
+      } else {
+        switch (readType) {
+          case FIXED_LENGTH_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedLengthDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case INT_LONG_BACKED_DECIMAL:
+            ((IcebergArrowVectors.DecimalArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchIntLongBackedDecimal(vec, typeWidth, nullabilityHolder);
+            break;
+          case VARBINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case VARCHAR:
+            ((IcebergArrowVectors.VarcharArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchVarWidthType(vec, nullabilityHolder);
+            break;
+          case FIXED_WIDTH_BINARY:
+            ((IcebergArrowVectors.VarBinaryArrowVector) vec).setNullabilityHolder(nullabilityHolder);
+            vectorizedColumnIterator.nextBatchFixedWidthBinary(vec, typeWidth, nullabilityHolder);
+            break;
+          case BOOLEAN:
+            vectorizedColumnIterator.nextBatchBoolean(vec, nullabilityHolder);
+            break;
+          case INT:
+            vectorizedColumnIterator.nextBatchIntegers(vec, typeWidth, nullabilityHolder);
+            break;
+          case LONG:
+            vectorizedColumnIterator.nextBatchLongs(vec, typeWidth, nullabilityHolder);
+            break;
+          case FLOAT:
+            vectorizedColumnIterator.nextBatchFloats(vec, typeWidth, nullabilityHolder);
+            break;
+          case DOUBLE:
+            vectorizedColumnIterator.nextBatchDoubles(vec, typeWidth, nullabilityHolder);
+            break;
+        }
+      }
+    }
+    Preconditions.checkState(vec.getValueCount() == numValsToRead,
+        "Number of values read, %s, does not equal expected, %s", vec.getValueCount(), numValsToRead);
+    return new VectorHolder(columnDescriptor, vec, allPagesDictEncoded, dictionary, nullabilityHolder);
+  }
+
+  private void allocateFieldVector() {
+    if (allPagesDictEncoded) {
+      Field field = new Field(
+          icebergField.name(),
+          new FieldType(icebergField.isOptional(), new ArrowType.Int(Integer.SIZE, true), null, null),
+          null);
+      this.vec = field.createVector(rootAlloc);
+      ((IntVector) vec).allocateNew(batchSize);
+      typeWidth = IntVector.TYPE_WIDTH;
+    } else {
+      PrimitiveType primitive = columnDescriptor.getPrimitiveType();
+      if (primitive.getOriginalType() != null) {
+        switch (columnDescriptor.getPrimitiveType().getOriginalType()) {
+          case ENUM:
+          case JSON:
+          case UTF8:
+          case BSON:
+            this.vec = new IcebergArrowVectors.VarcharArrowVector(icebergField.name(), rootAlloc);
+            //TODO: Possibly use the uncompressed page size info to set the initial capacity
+            vec.setInitialCapacity(batchSize * 10);
+            vec.allocateNewSafe();
+            readType = ReadType.VARCHAR;
+            typeWidth = UNKNOWN_WIDTH;
+            break;
+          case INT_8:
+          case INT_16:
+          case INT_32:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((IntVector) vec).allocateNew(batchSize);
+            readType = ReadType.INT;
+            typeWidth = IntVector.TYPE_WIDTH;
+            break;
+          case DATE:
+            this.vec = ArrowSchemaUtil.convert(icebergField).createVector(rootAlloc);
+            ((DateDayVector) vec).allocateNew(batchSize);
+            readType = ReadType.INT;
+            typeWidth = IntVector.TYPE_WIDTH;
+            break;
+          case INT_64:
+          case TIMESTAMP_MILLIS:
 
 Review comment:
   Why are millisecond timestamps read as longs instead of being converted to microseconds?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387190937
 
 

 ##########
 File path: parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java
 ##########
 @@ -0,0 +1,202 @@
+/*
+ * 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.parquet;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesInput;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.Encoding;
+import org.apache.parquet.column.ValuesType;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.DataPageV1;
+import org.apache.parquet.column.page.DataPageV2;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
+import org.apache.parquet.io.ParquetDecodingException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@SuppressWarnings("checkstyle:VisibilityModifier")
+public abstract class BasePageIterator {
+  private static final Logger LOG = LoggerFactory.getLogger(BasePageIterator.class);
+
+  protected final ColumnDescriptor desc;
+  protected final String writerVersion;
+
+  // iterator state
+  protected boolean hasNext = false;
+  protected int triplesRead = 0;
+  protected int currentDL = 0;
+  protected int currentRL = 0;
+
+  // page bookkeeping
+  protected Dictionary dictionary = null;
+  protected DataPage page = null;
+  protected int triplesCount = 0;
+  protected Encoding valueEncoding = null;
+  protected IntIterator definitionLevels = null;
+  protected IntIterator repetitionLevels = null;
+  protected ValuesReader vectorizedDefinitionLevelReader = null;
+  protected ValuesReader values = null;
+
+  protected BasePageIterator(ColumnDescriptor descriptor, String writerVersion) {
+    this.desc = descriptor;
+    this.writerVersion = writerVersion;
+  }
+
+  protected abstract void reset();
+
+  protected abstract boolean supportsVectorizedReads();
+
+  protected abstract IntIterator newNonVectorizedDefinitionLevelReader(ValuesReader dlReader);
+
+  protected abstract ValuesReader newVectorizedDefinitionLevelReader(ColumnDescriptor descriptor);
+
+  protected abstract void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount);
+
+  public void setPage(DataPage page) {
+    Preconditions.checkNotNull(page, "Cannot read from null page");
+    this.page = page;
+    this.page.accept(new DataPage.Visitor<ValuesReader>() {
+      @Override
+      public ValuesReader visit(DataPageV1 dataPageV1) {
+        initFromPage(dataPageV1);
+        return null;
+      }
+
+      @Override
+      public ValuesReader visit(DataPageV2 dataPageV2) {
+        initFromPage(dataPageV2);
+        return null;
+      }
+    });
+    this.triplesRead = 0;
+    this.hasNext = triplesRead < triplesCount;
+  }
+
+  protected void initFromPage(DataPageV1 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    ValuesReader dlReader = null;
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      dlReader = initPage.getDlEncoding().getValuesReader(desc, ValuesType.DEFINITION_LEVEL);
+      this.definitionLevels = newNonVectorizedDefinitionLevelReader(dlReader);
+    }
+    ValuesReader rlReader = initPage.getRlEncoding().getValuesReader(desc, ValuesType.REPETITION_LEVEL);
+    this.repetitionLevels = new PageIterator.ValuesReaderIntIterator(rlReader);
+    try {
+      BytesInput bytes = initPage.getBytes();
+      LOG.debug("page size {} bytes and {} records", bytes.size(), triplesCount);
+      LOG.debug("reading repetition levels at 0");
+      ByteBufferInputStream in = bytes.toInputStream();
+      rlReader.initFromPage(triplesCount, in);
+      LOG.debug("reading definition levels at {}", in.position());
+      if (supportsVectorizedReads()) {
+        this.vectorizedDefinitionLevelReader.initFromPage(triplesCount, in);
+      } else {
+        dlReader.initFromPage(triplesCount, in);
+      }
+      LOG.debug("reading data at {}", in.position());
+      initDataReader(initPage.getValueEncoding(), in, initPage.getValueCount());
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  protected void initFromPage(DataPageV2 initPage) {
+    this.triplesCount = initPage.getValueCount();
+    this.repetitionLevels = newRLEIterator(desc.getMaxRepetitionLevel(), initPage.getRepetitionLevels());
+    if (supportsVectorizedReads()) {
+      this.vectorizedDefinitionLevelReader = newVectorizedDefinitionLevelReader(desc);
+    } else {
+      this.definitionLevels = newRLEIterator(desc.getMaxDefinitionLevel(), initPage.getDefinitionLevels());
+    }
+    LOG.debug("page data size {} bytes and {} records", initPage.getData().size(), triplesCount);
+    try {
+      initDataReader(initPage.getDataEncoding(), initPage.getData().toInputStream(), triplesCount);
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read page " + initPage + " in col " + desc, e);
+    }
+  }
+
+  private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
+    try {
+      if (maxLevel == 0) {
+        return new NullIntIterator();
+      }
+      return new RLEIntIterator(
+          new RunLengthBitPackingHybridDecoder(
+              BytesUtils.getWidthFromMaxInt(maxLevel),
+              bytes.toInputStream()));
+    } catch (IOException e) {
+      throw new ParquetDecodingException("could not read levels in page for col " + desc, e);
+    }
+  }
+
+  public void setDictionary(Dictionary dict) {
+    this.dictionary = dict;
+  }
+
+  protected abstract static class IntIterator {
+    abstract int nextInt();
+  }
+
+  static class ValuesReaderIntIterator extends IntIterator {
 
 Review comment:
   Maybe move this into the non-vectorized version then?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r387371587
 
 

 ##########
 File path: versions.lock
 ##########
 @@ -95,9 +96,9 @@ org.antlr:antlr4-runtime:4.7 (1 constraints: 7a0e125f)
 org.antlr:stringtemplate:3.2.1 (1 constraints: c10a3bc6)
 org.apache.ant:ant:1.9.1 (3 constraints: a721ed14)
 org.apache.ant:ant-launcher:1.9.1 (1 constraints: 69082485)
-org.apache.arrow:arrow-format:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-memory:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-vector:0.10.0 (1 constraints: e90c9734)
+org.apache.arrow:arrow-format:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-memory:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-vector:0.14.1 (2 constraints: 2012a545)
 
 Review comment:
   @samarthjain, as a follow-up, we should open a PR for Arrow that fixes the DecimalVector nullability problem so we don't need an implementation here in Iceberg.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367668207
 
 

 ##########
 File path: arrow/src/test/java/org/apache/iceberg/arrow/ArrowSchemaUtilTest.java
 ##########
 @@ -0,0 +1,113 @@
+/*
+ * 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;
+
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.BooleanType;
+import org.apache.iceberg.types.Types.DateType;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.iceberg.types.Types.TimestampType;
+import org.junit.Test;
+
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Bool;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Date;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.FloatingPoint;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Int;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.List;
+import static org.apache.arrow.vector.types.pojo.ArrowType.ArrowTypeID.Timestamp;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+
+public class ArrowSchemaUtilTest {
+
+  @Test
+  public void convertPrimitive() {
+    Schema iceberg = new Schema(
+        optional(0, "i", Types.IntegerType.get()),
+        optional(1, "b", BooleanType.get()),
+        required(2, "d", DoubleType.get()),
+        required(3, "s", StringType.get()),
+        optional(4, "d2", DateType.get()),
+        optional(5, "ts", TimestampType.withoutZone())
 
 Review comment:
   Can we test `withZone` instead?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367631850
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
 
 Review comment:
   Style: we don't use `final`, we let the compiler infer it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
samarthjain commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r368105495
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
 
 Review comment:
   ```shouldAdjustToUTC``` is an instance level method in ```TimestampType```. Not sure how to apply the check here. 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue merged pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367657730
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
+  public static final int DEFAULT_BATCH_SIZE = 5000;
+  public static final int UNKNOWN_WIDTH = -1;
+
+  private final ColumnDescriptor columnDescriptor;
+  private final int batchSize;
+  private final VectorizedColumnIterator vectorizedColumnIterator;
+  private final boolean isFixedLengthDecimal;
+  private final boolean isVarWidthType;
+  private final boolean isFixedWidthBinary;
+  private final boolean isBooleanType;
+  private final boolean isPaddedDecimal;
+  private final boolean isIntType;
+  private final boolean isLongType;
+  private final boolean isFloatType;
+  private final boolean isDoubleType;
+  private final Types.NestedField icebergField;
+  private final BufferAllocator rootAlloc;
+  private FieldVector vec;
+  private int typeWidth;
+  private boolean reuseContainers = true;
+  private NullabilityHolder nullabilityHolder;
+
+  // In cases when Parquet employs fall back to plain encoding, we eagerly decode the dictionary encoded pages
+  // before storing the values in the Arrow vector. This means even if the dictionary is present, data
+  // present in the vector may not necessarily be dictionary encoded.
+  private Dictionary dictionary;
+  private boolean allPagesDictEncoded;
+
+  // This value is copied from Arrow's BaseVariableWidthVector. We may need to change
+  // this value if Arrow ends up changing this default.
+  private static final int DEFAULT_RECORD_BYTE_COUNT = 8;
+
+  public VectorizedArrowReader(
+      ColumnDescriptor desc,
+      Types.NestedField icebergField,
+      BufferAllocator ra,
+      int batchSize) {
+    this.icebergField = icebergField;
+    this.batchSize = (batchSize == 0) ? DEFAULT_BATCH_SIZE : batchSize;
+    this.columnDescriptor = desc;
+    this.rootAlloc = ra;
+    this.isFixedLengthDecimal = isFixedLengthDecimal(desc);
+    this.isVarWidthType = isVarWidthType(desc);
+    this.isFixedWidthBinary = isFixedWidthBinary(desc);
+    this.isBooleanType = isBooleanType(desc);
+    this.isPaddedDecimal = isIntLongBackedDecimal(desc);
+    this.isIntType = isIntType(desc);
+    this.isLongType = isLongType(desc);
+    this.isFloatType = isFloatType(desc);
+    this.isDoubleType = isDoubleType(desc);
 
 Review comment:
   Would it make sense to use an enum instead of 9 booleans?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367731909
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] decimalBytes = dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe();
+              byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+              System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP) This
+   * method reads batches of bytes from Parquet and writes them into the data buffer underneath the Arrow vector. It
+   * appropriately sets the validity buffer in the Arrow vector.
+   */
+  public void readBatchVarWidth(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setVarWidthBinaryValue(FieldVector vector, ValuesAsBytesReader valuesReader, int bufferIdx) {
+    int len = valuesReader.readInteger();
+    ByteBuffer buffer = valuesReader.getBuffer(len);
+    // Calling setValueLengthSafe takes care of allocating a larger buffer if
+    // running out of space.
+    ((BaseVariableWidthVector) vector).setValueLengthSafe(bufferIdx, len);
+    // It is possible that the data buffer was reallocated. So it is important to
+    // not cache the data buffer reference but instead use vector.getDataBuffer().
+    vector.getDataBuffer().writeBytes(buffer.array(), buffer.position(), buffer.limit() - buffer.position());
 
 Review comment:
   I don't think this is safe. Don't you need to use `offset = buffer.position() + buffer.arrayOffset()` when accessing the backing array?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367654007
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedArrowReader.java
 ##########
 @@ -0,0 +1,387 @@
+/*
+ * 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;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.arrow.ArrowSchemaUtil;
+import org.apache.iceberg.arrow.vectorized.parquet.VectorizedColumnIterator;
+import org.apache.iceberg.parquet.ParquetUtil;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.schema.DecimalMetadata;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/***
+ * {@link VectorizedReader VectorReader(s)} that read in a batch of values into Arrow vectors.
+ * It also takes care of allocating the right kind of Arrow vectors depending on the corresponding
+ * Iceberg/Parquet data types.
+ */
+public class VectorizedArrowReader implements VectorizedReader<VectorHolder> {
 
 Review comment:
   Is any of this from Spark or based on code from Spark? If so, we should note in comments where it came from and add a list of affected files to LICENSE.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367723389
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
 
 Review comment:
   Should this be called `offset` since it is the offset to copy into the output vector?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367642235
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/ArrowSchemaUtil.java
 ##########
 @@ -0,0 +1,147 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.types.pojo.Schema;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.ListType;
+import org.apache.iceberg.types.Types.MapType;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.types.Types.StructType;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+
+public class ArrowSchemaUtil {
+  static final String ORIGINAL_TYPE = "originalType";
+  static final String MAP_TYPE = "mapType";
+  static final String MAP_KEY = "key";
+  static final String MAP_VALUE = "value";
+
+  private ArrowSchemaUtil() { }
+
+  /**
+   * Convert Iceberg schema to Arrow Schema.
+   *
+   * @param schema iceberg schema
+   * @return arrow schema
+   */
+  public static Schema convert(final org.apache.iceberg.Schema schema) {
+    final ImmutableList.Builder<Field> fields = ImmutableList.builder();
+
+    for (NestedField f : schema.columns()) {
+      fields.add(convert(f));
+    }
+
+    return new Schema(fields.build());
+  }
+
+  public static Field convert(final NestedField field) {
+    final ArrowType arrowType;
+
+    final List<Field> children = Lists.newArrayList();
+    Map<String, String> metadata = null;
+
+    switch (field.type().typeId()) {
+      case BINARY:
+        // Spark doesn't support BYTE(fixed_size) type, so cast it to VarBinary
+      case FIXED:
+        arrowType = ArrowType.Binary.INSTANCE;
+        break;
+      case BOOLEAN:
+        arrowType = ArrowType.Bool.INSTANCE;
+        break;
+      case INTEGER:
+        arrowType = new ArrowType.Int(Integer.SIZE, true);
+        break;
+      case LONG:
+        arrowType = new ArrowType.Int(Long.SIZE, true);
+        break;
+      case FLOAT:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE);
+        break;
+      case DOUBLE:
+        arrowType = new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE);
+        break;
+      case DECIMAL:
+        final Types.DecimalType decimalType = (Types.DecimalType) field.type();
+        arrowType = new ArrowType.Decimal(decimalType.precision(), decimalType.scale());
+        break;
+      case STRING:
+        arrowType = ArrowType.Utf8.INSTANCE;
+        break;
+      case TIME:
+        arrowType = new ArrowType.Time(TimeUnit.MICROSECOND, Long.SIZE);
+        break;
+      case TIMESTAMP:
+        arrowType = new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC");
+        break;
+      case DATE:
+        arrowType = new ArrowType.Date(DateUnit.DAY);
+        break;
+      case STRUCT:
+        final StructType struct = field.type().asStructType();
+        arrowType = ArrowType.Struct.INSTANCE;
+
+        for (NestedField nested : struct.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case LIST:
+        final ListType listType = field.type().asListType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        for (NestedField nested : listType.fields()) {
+          children.add(convert(nested));
+        }
+        break;
+      case MAP:
+        //Maps are represented as List<Struct<key, value>>
+        metadata = ImmutableMap.of(ORIGINAL_TYPE, MAP_TYPE);
+        final MapType mapType = field.type().asMapType();
+        arrowType = ArrowType.List.INSTANCE;
+
+        final List<Field> entryFields = Lists.newArrayList(
+            convert(required(0, MAP_KEY, mapType.keyType())),
+            convert(optional(0, MAP_VALUE, mapType.valueType()))
+        );
+
+        final Field entry = new Field("",
+            new FieldType(true, new ArrowType.Struct(), null), entryFields);
 
 Review comment:
   Why does this use `new ArrowType.Struct()` instead of using `ArrowType.Struct.INSTANCE` that is used above?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #723: Arrow changes for supporting vectorized reads
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r367728615
 
 

 ##########
 File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/VectorizedParquetValuesReader.java
 ##########
 @@ -0,0 +1,1375 @@
+/*
+ * 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 io.netty.buffer.ArrowBuf;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.arrow.vector.BaseVariableWidthVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.BitVectorHelper;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.iceberg.arrow.vectorized.NullabilityHolder;
+import org.apache.iceberg.parquet.ValuesAsBytesReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.bytes.ByteBufferInputStream;
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.column.values.ValuesReader;
+import org.apache.parquet.column.values.bitpacking.BytePacker;
+import org.apache.parquet.column.values.bitpacking.Packer;
+import org.apache.parquet.io.ParquetDecodingException;
+
+/**
+ * A values reader for Parquet's run-length encoded data that reads column data in batches instead of one value at a
+ * time. This is based off of the version in Apache Spark with these changes:
+ * <p>
+ * <tr>Writes batches of values retrieved to Arrow vectors</tr>
+ * <tr>If all pages of a column within the row group are not dictionary encoded, then
+ * dictionary ids are eagerly decoded into actual values before writing them to the Arrow vectors</tr>
+ * </p>
+ */
+public final class VectorizedParquetValuesReader extends ValuesReader {
+
+  // Current decoding mode. The encoded data contains groups of either run length encoded data
+  // (RLE) or bit packed data. Each group contains a header that indicates which group it is and
+  // the number of values in the group.
+  private enum MODE {
+    RLE,
+    PACKED
+  }
+
+  // Encoded data.
+  private ByteBufferInputStream inputStream;
+
+  // bit/byte width of decoded data and utility to batch unpack them.
+  private int bitWidth;
+  private int bytesWidth;
+  private BytePacker packer;
+
+  // Current decoding mode and values
+  private MODE mode;
+  private int currentCount;
+  private int currentValue;
+
+  // Buffer of decoded values if the values are PACKED.
+  private int[] packedValuesBuffer = new int[16];
+  private int packedValuesBufferIdx = 0;
+
+  // If true, the bit width is fixed. This decoder is used in different places and this also
+  // controls if we need to read the bitwidth from the beginning of the data stream.
+  private final boolean fixedWidth;
+  private final boolean readLength;
+  private final int maxDefLevel;
+
+  public VectorizedParquetValuesReader(int maxDefLevel) {
+    this.maxDefLevel = maxDefLevel;
+    this.fixedWidth = false;
+    this.readLength = false;
+  }
+
+  public VectorizedParquetValuesReader(
+      int bitWidth,
+      int maxDefLevel) {
+    this.fixedWidth = true;
+    this.readLength = bitWidth != 0;
+    this.maxDefLevel = maxDefLevel;
+    init(bitWidth);
+  }
+
+  public VectorizedParquetValuesReader(
+      int bw,
+      boolean rl,
+      int mdl) {
+    this.fixedWidth = true;
+    this.readLength = rl;
+    this.maxDefLevel = mdl;
+    init(bw);
+  }
+
+  @Override
+  public void initFromPage(int valueCount, ByteBufferInputStream in) throws IOException {
+    this.inputStream = in;
+    if (fixedWidth) {
+      // initialize for repetition and definition levels
+      if (readLength) {
+        int length = readIntLittleEndian();
+        this.inputStream = in.sliceStream(length);
+      }
+    } else {
+      // initialize for values
+      if (in.available() > 0) {
+        init(in.read());
+      }
+    }
+    if (bitWidth == 0) {
+      // 0 bit width, treat this as an RLE run of valueCount number of 0's.
+      this.mode = MODE.RLE;
+      this.currentCount = valueCount;
+      this.currentValue = 0;
+    } else {
+      this.currentCount = 0;
+    }
+  }
+
+  /**
+   * Initializes the internal state for decoding ints of `bitWidth`.
+   */
+  private void init(int bw) {
+    Preconditions.checkArgument(bw >= 0 && bw <= 32, "bitWidth must be >= 0 and <= 32");
+    this.bitWidth = bw;
+    this.bytesWidth = BytesUtils.paddedByteCountFromBits(bw);
+    this.packer = Packer.LITTLE_ENDIAN.newBytePacker(bw);
+  }
+
+  /**
+   * Reads the next varint encoded int.
+   */
+  private int readUnsignedVarInt() throws IOException {
+    int value = 0;
+    int shift = 0;
+    int byteRead;
+    do {
+      byteRead = inputStream.read();
+      value |= (byteRead & 0x7F) << shift;
+      shift += 7;
+    } while ((byteRead & 0x80) != 0);
+    return value;
+  }
+
+  /**
+   * Reads the next 4 byte little endian int.
+   */
+  private int readIntLittleEndian() throws IOException {
+    int ch4 = inputStream.read();
+    int ch3 = inputStream.read();
+    int ch2 = inputStream.read();
+    int ch1 = inputStream.read();
+    return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0);
+  }
+
+  /**
+   * Reads the next byteWidth little endian int.
+   */
+  private int readIntLittleEndianPaddedOnBitWidth() throws IOException {
+    switch (bytesWidth) {
+      case 0:
+        return 0;
+      case 1:
+        return inputStream.read();
+      case 2: {
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 8) + ch2;
+      }
+      case 3: {
+        int ch3 = inputStream.read();
+        int ch2 = inputStream.read();
+        int ch1 = inputStream.read();
+        return (ch1 << 16) + (ch2 << 8) + (ch3 << 0);
+      }
+      case 4: {
+        return readIntLittleEndian();
+      }
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  /**
+   * Reads the next group.
+   */
+  private void readNextGroup() {
+    try {
+      int header = readUnsignedVarInt();
+      this.mode = (header & 1) == 0 ? MODE.RLE : MODE.PACKED;
+      switch (mode) {
+        case RLE:
+          this.currentCount = header >>> 1;
+          this.currentValue = readIntLittleEndianPaddedOnBitWidth();
+          return;
+        case PACKED:
+          int numGroups = header >>> 1;
+          this.currentCount = numGroups * 8;
+
+          if (this.packedValuesBuffer.length < this.currentCount) {
+            this.packedValuesBuffer = new int[this.currentCount];
+          }
+          packedValuesBufferIdx = 0;
+          int valueIndex = 0;
+          while (valueIndex < this.currentCount) {
+            // values are bit packed 8 at a time, so reading bitWidth will always work
+            ByteBuffer buffer = inputStream.slice(bitWidth);
+            this.packer.unpack8Values(buffer, buffer.position(), this.packedValuesBuffer, valueIndex);
+            valueIndex += 8;
+          }
+          return;
+        default:
+          throw new ParquetDecodingException("not a valid mode " + this.mode);
+      }
+    } catch (IOException e) {
+      throw new ParquetDecodingException("Failed to read from input stream", e);
+    }
+  }
+
+  @Override
+  public boolean readBoolean() {
+    return this.readInteger() != 0;
+  }
+
+  @Override
+  public void skip() {
+    this.readInteger();
+  }
+
+  @Override
+  public int readValueDictionaryId() {
+    return readInteger();
+  }
+
+  @Override
+  public int readInteger() {
+    if (this.currentCount == 0) {
+      this.readNextGroup();
+    }
+
+    this.currentCount--;
+    switch (mode) {
+      case RLE:
+        return this.currentValue;
+      case PACKED:
+        return this.packedValuesBuffer[packedValuesBufferIdx++];
+    }
+    throw new RuntimeException("Unreachable");
+  }
+
+  public void readBatchOfDictionaryIds(
+      final IntVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readDictionaryIdsInternal(vector, idx, numValues);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, vector.getValidityBuffer());
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.set(idx, dictionaryEncodedValuesReader.readInteger());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  // Used for reading dictionary ids in a vectorized fashion. Unlike other methods, this doesn't
+  // check definition level.
+  private void readDictionaryIdsInternal(
+      final IntVector intVector,
+      final int numValsInVector,
+      final int numValuesToRead) {
+    int left = numValuesToRead;
+    int idx = numValsInVector;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, currentValue);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            intVector.set(idx, packedValuesBuffer[packedValuesBufferIdx]);
+            packedValuesBufferIdx++;
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfLongs(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              numValues);
+          bufferIdx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(
+                  typeWidth,
+                  valuesReader,
+                  bufferIdx,
+                  vector.getValidityBuffer(),
+                  vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedLongs(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedLongsInternal(vector, typeWidth, idx, numValues, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, numValues, validityBuffer);
+          }
+          idx += numValues;
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setLong(idx, dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedLongsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int numValues = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer().setLong(idx, dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < numValues; i++) {
+            vector.getDataBuffer()
+                .setLong(idx, dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= numValues;
+      currentCount -= numValues;
+    }
+  }
+
+  public void readBatchOfIntegers(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntegers(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntegersInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setInt(idx, dict.decodeToInt(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntegersInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf dataBuffer = vector.getDataBuffer();
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            dataBuffer.setInt(idx, dict.decodeToInt(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFloats(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder, ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setValue(
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx,
+      ArrowBuf validityBuffer,
+      ArrowBuf dataBuffer) {
+    dataBuffer.setBytes(bufferIdx * typeWidth, valuesReader.getBuffer(typeWidth));
+    BitVectorHelper.setValidityBitToOne(validityBuffer, bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedFloats(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      ArrowBuf validityBuffer = vector.getValidityBuffer();
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFloatsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, validityBuffer);
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, validityBuffer);
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFloatsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setFloat(idx, dict.decodeToFloat(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDoubles(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          setNextNValuesInVector(
+              typeWidth,
+              nullabilityHolder,
+              valuesReader,
+              bufferIdx,
+              vector,
+              num);
+          bufferIdx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setValue(typeWidth, valuesReader, bufferIdx, vector.getValidityBuffer(), vector.getDataBuffer());
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedDoubles(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedDoublesInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedDoublesInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setDouble(idx, dict.decodeToDouble(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedWidthBinary(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setBinaryInVector((VarBinaryVector) vector, typeWidth, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedWidthBinary(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedWidthBinaryInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              vector.getDataBuffer()
+                  .setBytes(idx * typeWidth, dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedWidthBinaryInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer().setBytes(idx * typeWidth, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            vector.getDataBuffer()
+                .setBytes(
+                    idx * typeWidth,
+                    dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), idx);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfFixedLengthDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(bufferIdx, byteArray);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedFixedLengthDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] decimalBytes = dict.decodeToBinary(valuesReader.readInteger()).getBytesUnsafe();
+              byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+              System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+              ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedFixedLengthDecimalsInternal(
+      FieldVector vector,
+      int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(currentValue).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            byte[] decimalBytes = dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe();
+            byte[] vectorBytes = new byte[DecimalVector.TYPE_WIDTH];
+            System.arraycopy(decimalBytes, 0, vectorBytes, DecimalVector.TYPE_WIDTH - typeWidth, typeWidth);
+            ((DecimalVector) vector).setBigEndian(idx, vectorBytes);
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  /**
+   * Method for reading a batch of non-decimal numeric data types (INT32, INT64, FLOAT, DOUBLE, DATE, TIMESTAMP) This
+   * method reads batches of bytes from Parquet and writes them into the data buffer underneath the Arrow vector. It
+   * appropriately sets the validity buffer in the Arrow vector.
+   */
+  public void readBatchVarWidth(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              setVarWidthBinaryValue(vector, valuesReader, bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setVarWidthBinaryValue(FieldVector vector, ValuesAsBytesReader valuesReader, int bufferIdx) {
+    int len = valuesReader.readInteger();
+    ByteBuffer buffer = valuesReader.getBuffer(len);
+    // Calling setValueLengthSafe takes care of allocating a larger buffer if
+    // running out of space.
+    ((BaseVariableWidthVector) vector).setValueLengthSafe(bufferIdx, len);
+    // It is possible that the data buffer was reallocated. So it is important to
+    // not cache the data buffer reference but instead use vector.getDataBuffer().
+    vector.getDataBuffer().writeBytes(buffer.array(), buffer.position(), buffer.limit() - buffer.position());
+    // Similarly, we need to get the latest reference to the validity buffer as well
+    // since reallocation changes reference of the validity buffers as well.
+    BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+  }
+
+  public void readBatchOfDictionaryEncodedVarWidth(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader dictionaryEncodedValuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            dictionaryEncodedValuesReader.readBatchOfDictionaryEncodedVarWidthBinaryInternal(vector, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((BaseVariableWidthVector) vector).setSafe(
+                  idx,
+                  dict.decodeToBinary(dictionaryEncodedValuesReader.readInteger()).getBytesUnsafe());
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedVarWidthBinaryInternal(
+      FieldVector vector,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            ((BaseVariableWidthVector) vector).setSafe(idx, dict.decodeToBinary(currentValue).getBytesUnsafe());
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            ((BaseVariableWidthVector) vector).setSafe(
+                idx,
+                dict.decodeToBinary(packedValuesBuffer[packedValuesBufferIdx++]).getBytesUnsafe());
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfIntLongBackedDecimals(
+      final FieldVector vector, final int numValsInVector,
+      final int typeWidth, final int batchSize, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth);
+              vector.getDataBuffer().setBytes(bufferIdx * DecimalVector.TYPE_WIDTH, byteArray);
+              BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              byte[] byteArray = new byte[DecimalVector.TYPE_WIDTH];
+              valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth);
+              vector.getDataBuffer().setBytes(bufferIdx * DecimalVector.TYPE_WIDTH, byteArray);
+              BitVectorHelper.setValidityBitToOne(vector.getValidityBuffer(), bufferIdx);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfDictionaryEncodedIntLongBackedDecimals(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int typeWidth,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      VectorizedParquetValuesReader valuesReader,
+      Dictionary dict) {
+    int idx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            valuesReader.readBatchOfDictionaryEncodedIntLongBackedDecimalsInternal(vector, typeWidth, idx, num, dict);
+          } else {
+            setNulls(nullabilityHolder, idx, num, vector.getValidityBuffer());
+          }
+          idx += num;
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((DecimalVector) vector).set(
+                  idx,
+                  typeWidth == Integer.BYTES ?
+                      dict.decodeToInt(valuesReader.readInteger())
+                      : dict.decodeToLong(valuesReader.readInteger()));
+            } else {
+              setNull(nullabilityHolder, idx, vector.getValidityBuffer());
+            }
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void readBatchOfDictionaryEncodedIntLongBackedDecimalsInternal(
+      FieldVector vector,
+      final int typeWidth,
+      int index,
+      int numValuesToRead,
+      Dictionary dict) {
+    int left = numValuesToRead;
+    int idx = index;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          for (int i = 0; i < num; i++) {
+            ((DecimalVector) vector).set(
+                idx,
+                typeWidth == Integer.BYTES ? dict.decodeToInt(currentValue) : dict.decodeToLong(currentValue));
+            idx++;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; i++) {
+            ((DecimalVector) vector).set(
+                idx,
+                typeWidth == Integer.BYTES ?
+                    dict.decodeToInt(currentValue)
+                    : dict.decodeToLong(packedValuesBuffer[packedValuesBufferIdx++]));
+            idx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  public void readBatchOfBooleans(
+      final FieldVector vector,
+      final int numValsInVector,
+      final int batchSize,
+      NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader) {
+    int bufferIdx = numValsInVector;
+    int left = batchSize;
+    while (left > 0) {
+      if (this.currentCount == 0) {
+        this.readNextGroup();
+      }
+      int num = Math.min(left, this.currentCount);
+      switch (mode) {
+        case RLE:
+          if (currentValue == maxDefLevel) {
+            for (int i = 0; i < num; i++) {
+              ((BitVector) vector).setSafe(bufferIdx, valuesReader.readBoolean() ? 1 : 0);
+              bufferIdx++;
+            }
+          } else {
+            setNulls(nullabilityHolder, bufferIdx, num, vector.getValidityBuffer());
+            bufferIdx += num;
+          }
+          break;
+        case PACKED:
+          for (int i = 0; i < num; ++i) {
+            if (packedValuesBuffer[packedValuesBufferIdx++] == maxDefLevel) {
+              ((BitVector) vector).setSafe(bufferIdx, valuesReader.readBoolean() ? 1 : 0);
+            } else {
+              setNull(nullabilityHolder, bufferIdx, vector.getValidityBuffer());
+            }
+            bufferIdx++;
+          }
+          break;
+      }
+      left -= num;
+      currentCount -= num;
+    }
+  }
+
+  private void setBinaryInVector(
+      VarBinaryVector vector,
+      int typeWidth,
+      ValuesAsBytesReader valuesReader,
+      int bufferIdx) {
+    byte[] byteArray = new byte[typeWidth];
+    valuesReader.getBuffer(typeWidth).get(byteArray);
+    vector.setSafe(bufferIdx, byteArray);
+  }
+
+  private void setNextNValuesInVector(
+      int typeWidth, NullabilityHolder nullabilityHolder,
+      ValuesAsBytesReader valuesReader, int bufferIdx, FieldVector vector, int numValues) {
+    ArrowBuf validityBuffer = vector.getValidityBuffer();
+    int validityBufferIdx = bufferIdx;
+    if (currentValue == maxDefLevel) {
+      for (int i = 0; i < numValues; i++) {
+        BitVectorHelper.setValidityBitToOne(validityBuffer, validityBufferIdx);
+        validityBufferIdx++;
+      }
+      ByteBuffer buffer = valuesReader.getBuffer(numValues * typeWidth);
+      vector.getDataBuffer().setBytes(bufferIdx * typeWidth, buffer);
+    } else {
+      setNulls(nullabilityHolder, bufferIdx, numValues, validityBuffer);
+    }
+  }
+
+  private void setNull(NullabilityHolder nullabilityHolder, int bufferIdx, ArrowBuf validityBuffer) {
+    nullabilityHolder.setNull(bufferIdx);
+    BitVectorHelper.setValidityBit(validityBuffer, bufferIdx, 0);
+  }
+
+  private void setNulls(NullabilityHolder nullabilityHolder, int idx, int numValues, ArrowBuf validityBuffer) {
+    int bufferIdx = idx;
+    for (int i = 0; i < numValues; i++) {
 
 Review comment:
   Is it better to set both buffers at the same time, or better to loop twice?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [incubator-iceberg] jacques-n commented on a change in pull request #723: Arrow changes for supporting vectorized reads

Posted by GitBox <gi...@apache.org>.
jacques-n commented on a change in pull request #723:
URL: https://github.com/apache/incubator-iceberg/pull/723#discussion_r411846205



##########
File path: versions.lock
##########
@@ -95,9 +96,9 @@ org.antlr:antlr4-runtime:4.7 (1 constraints: 7a0e125f)
 org.antlr:stringtemplate:3.2.1 (1 constraints: c10a3bc6)
 org.apache.ant:ant:1.9.1 (3 constraints: a721ed14)
 org.apache.ant:ant-launcher:1.9.1 (1 constraints: 69082485)
-org.apache.arrow:arrow-format:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-memory:0.10.0 (1 constraints: 1f0de721)
-org.apache.arrow:arrow-vector:0.10.0 (1 constraints: e90c9734)
+org.apache.arrow:arrow-format:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-memory:0.14.1 (1 constraints: 240df421)
+org.apache.arrow:arrow-vector:0.14.1 (2 constraints: 2012a545)

Review comment:
       Yeah, it would be great to see an Arrow patch for what would be the appropriate fix.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org