diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java
index a0dced01e5e8d..2f21a323302f1 100644
--- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java
+++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapArrayVector.java
@@ -57,6 +57,37 @@ public int getLen() {
return this.isNull.length;
}
+ // ---------------------------------------------------------------------------------------------
+ // Flink 2.1-compatible accessors. Backed by the existing public {@code offsets}, {@code lengths}
+ // and {@code child} fields so legacy callers continue to work; the new {@link
+ // org.apache.hudi.table.format.cow.vector.reader.NestedColumnReader} (FLINK-35702 port) and any
+ // future Flink-2.1-style caller use these accessors.
+ // ---------------------------------------------------------------------------------------------
+
+ public long[] getOffsets() {
+ return offsets;
+ }
+
+ public void setOffsets(long[] offsets) {
+ this.offsets = offsets;
+ }
+
+ public long[] getLengths() {
+ return lengths;
+ }
+
+ public void setLengths(long[] lengths) {
+ this.lengths = lengths;
+ }
+
+ public ColumnVector getChild() {
+ return child;
+ }
+
+ public void setChild(ColumnVector child) {
+ this.child = child;
+ }
+
@Override
public ArrayData getArray(int i) {
long offset = offsets[i];
diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java
index 0d83f82baedf3..a98bdebd707a7 100644
--- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java
+++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapMapColumnVector.java
@@ -20,6 +20,7 @@
import lombok.Getter;
import org.apache.flink.table.data.MapData;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
import org.apache.flink.table.data.columnar.vector.MapColumnVector;
import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector;
import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
@@ -31,16 +32,74 @@ public class HeapMapColumnVector extends AbstractHeapVector
implements WritableColumnVector, MapColumnVector {
@Getter
- private final WritableColumnVector keys;
+ private WritableColumnVector keys;
@Getter
- private final WritableColumnVector values;
+ private WritableColumnVector values;
+
+ // ---------------------------------------------------------------------------------------------
+ // Flink 2.1 Dremel-style state. Populated by {@link
+ // org.apache.hudi.table.format.cow.vector.reader.NestedColumnReader} (FLINK-35702 port). The
+ // legacy {@link #getMap(int)} implementation below continues to use {@code ColumnarGroupMapData}
+ // — wiring it through these offsets/lengths happens in a follow-up PR that switches the read
+ // path. Left here so the new readers can compile against the additive surface.
+ // ---------------------------------------------------------------------------------------------
+ private long[] offsets;
+ private long[] lengths;
+ private int size;
public HeapMapColumnVector(int len, WritableColumnVector keys, WritableColumnVector values) {
super(len);
+ this.offsets = new long[len];
+ this.lengths = new long[len];
this.keys = keys;
this.values = values;
}
+ public long[] getOffsets() {
+ return offsets;
+ }
+
+ public void setOffsets(long[] offsets) {
+ this.offsets = offsets;
+ }
+
+ public long[] getLengths() {
+ return lengths;
+ }
+
+ public void setLengths(long[] lengths) {
+ this.lengths = lengths;
+ }
+
+ public int getSize() {
+ return size;
+ }
+
+ public void setSize(int size) {
+ this.size = size;
+ }
+
+ public void setKeys(WritableColumnVector keys) {
+ this.keys = keys;
+ }
+
+ public void setValues(WritableColumnVector values) {
+ this.values = values;
+ }
+
+ /**
+ * Returns the keys child vector typed as {@link ColumnVector}, matching the Flink 2.1 contract
+ * consumed by {@code NestedColumnReader}. Functionally equivalent to {@link #getKeys()}.
+ */
+ public ColumnVector getKeyColumnVector() {
+ return keys;
+ }
+
+ /** Counterpart of {@link #getKeyColumnVector()} for the values child vector. */
+ public ColumnVector getValueColumnVector() {
+ return values;
+ }
+
@Override
public MapData getMap(int rowId) {
return new ColumnarGroupMapData(keys, values, rowId);
diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java
index ae194e4e6ab05..0c640ce92ee40 100644
--- a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java
+++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/HeapRowColumnVector.java
@@ -37,6 +37,21 @@ public HeapRowColumnVector(int len, WritableColumnVector... vectors) {
this.vectors = vectors;
}
+ /**
+ * Flink 2.1-compatible accessor for the children vectors. Backed by the existing public {@code
+ * vectors} field so legacy callers continue to work; the new {@link
+ * org.apache.hudi.table.format.cow.vector.reader.NestedColumnReader} (FLINK-35702 port) and any
+ * future Flink-2.1-style caller use this accessor.
+ */
+ public WritableColumnVector[] getFields() {
+ return vectors;
+ }
+
+ /** Counterpart of {@link #getFields()}. */
+ public void setFields(WritableColumnVector[] fields) {
+ this.vectors = fields;
+ }
+
@Override
public ColumnarRowData getRow(int i) {
ColumnarRowData columnarRowData = new ColumnarRowData(new VectorizedColumnBatch(vectors));
diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/NestedColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/NestedColumnReader.java
new file mode 100644
index 0000000000000..27eab298b3207
--- /dev/null
+++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/NestedColumnReader.java
@@ -0,0 +1,257 @@
+/*
+ * 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.hudi.table.format.cow.vector.reader;
+
+import org.apache.hudi.table.format.cow.utils.NestedPositionUtil;
+import org.apache.hudi.table.format.cow.vector.HeapArrayVector;
+import org.apache.hudi.table.format.cow.vector.HeapMapColumnVector;
+import org.apache.hudi.table.format.cow.vector.HeapRowColumnVector;
+import org.apache.hudi.table.format.cow.vector.position.CollectionPosition;
+import org.apache.hudi.table.format.cow.vector.position.LevelDelegation;
+import org.apache.hudi.table.format.cow.vector.position.RowPosition;
+import org.apache.hudi.table.format.cow.vector.type.ParquetField;
+import org.apache.hudi.table.format.cow.vector.type.ParquetGroupField;
+import org.apache.hudi.table.format.cow.vector.type.ParquetPrimitiveField;
+
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
+import org.apache.flink.table.data.columnar.vector.ColumnVector;
+import org.apache.flink.table.data.columnar.vector.heap.AbstractHeapVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
+import org.apache.flink.table.types.logical.ArrayType;
+import org.apache.flink.table.types.logical.MapType;
+import org.apache.flink.table.types.logical.MultisetType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.Preconditions;
+
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.page.PageReadStore;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * ColumnReader used to read a {@code Group} type in Parquet ({@code Map}, {@code Array}, {@code
+ * Row}). Resolves nested structures using Dremel striping/assembly; see the
+ * striping and assembly algorithms from the Dremel paper.
+ *
+ *
Vendored from Apache Flink 2.1 (FLINK-35702, {@code
+ * org.apache.flink.formats.parquet.vector.reader.NestedColumnReader}). Differences vs. upstream:
+ *
+ *
+ *
Uses Hudi-local {@code HeapRowColumnVector}/{@code HeapMapColumnVector}/{@code
+ * HeapArrayVector} instead of the Flink-private {@code HeapRowVector}/{@code
+ * HeapMapVector}/{@code HeapArrayVector}.
+ *
Supports Hudi's schema-evolution contract: a {@code ParquetGroupField} representing a
+ * {@link RowType} may contain {@code null} children — meaning the corresponding logical
+ * field is absent from the Parquet file. Those slots are passed through unchanged and do
+ * not contribute to the row's repetition/definition-level stream.
+ *
+ */
+public class NestedColumnReader implements ColumnReader {
+
+ private final Map columnReaders;
+ private final boolean isUtcTimestamp;
+
+ private final PageReadStore pages;
+
+ private final ParquetField field;
+
+ public NestedColumnReader(boolean isUtcTimestamp, PageReadStore pages, ParquetField field) {
+ this.isUtcTimestamp = isUtcTimestamp;
+ this.pages = pages;
+ this.field = field;
+ this.columnReaders = new HashMap<>();
+ }
+
+ @Override
+ public void readToVector(int readNumber, WritableColumnVector vector) throws IOException {
+ readData(field, readNumber, vector, false);
+ }
+
+ private Tuple2 readData(
+ ParquetField field, int readNumber, ColumnVector vector, boolean inside) throws IOException {
+ if (field.getType() instanceof RowType) {
+ return readRow((ParquetGroupField) field, readNumber, vector, inside);
+ } else if (field.getType() instanceof MapType || field.getType() instanceof MultisetType) {
+ return readMap((ParquetGroupField) field, readNumber, vector, inside);
+ } else if (field.getType() instanceof ArrayType) {
+ return readArray((ParquetGroupField) field, readNumber, vector, inside);
+ } else {
+ return readPrimitive((ParquetPrimitiveField) field, readNumber, vector);
+ }
+ }
+
+ private Tuple2 readRow(
+ ParquetGroupField field, int readNumber, ColumnVector vector, boolean inside)
+ throws IOException {
+ HeapRowColumnVector heapRowVector = (HeapRowColumnVector) vector;
+ LevelDelegation levelDelegation = null;
+ List children = field.getChildren();
+ WritableColumnVector[] childrenVectors = heapRowVector.getFields();
+ WritableColumnVector[] finalChildrenVectors = new WritableColumnVector[childrenVectors.length];
+ for (int i = 0; i < children.size(); i++) {
+ ParquetField child = children.get(i);
+ if (child == null) {
+ // Hudi schema-evolution: the logical field is not present in the Parquet file. The slot
+ // vector is expected to be pre-populated with nulls by the caller (lands in a follow-up
+ // PR that rewires ParquetSplitReaderUtil); keep it as is and skip contributing to the
+ // level stream.
+ finalChildrenVectors[i] = childrenVectors[i];
+ continue;
+ }
+ Tuple2 tuple =
+ readData(child, readNumber, childrenVectors[i], true);
+ levelDelegation = tuple.f0;
+ finalChildrenVectors[i] = tuple.f1;
+ }
+ if (levelDelegation == null) {
+ throw new FlinkRuntimeException(
+ String.format("Row field does not have any non-null children: %s.", field));
+ }
+
+ RowPosition rowPosition =
+ NestedPositionUtil.calculateRowOffsets(
+ field,
+ levelDelegation.getDefinitionLevel(),
+ levelDelegation.getRepetitionLevel());
+
+ // If row was inside the structure, then we need to renew the vector to reset the
+ // capacity.
+ if (inside) {
+ heapRowVector = new HeapRowColumnVector(rowPosition.getPositionsCount(), finalChildrenVectors);
+ } else {
+ heapRowVector.setFields(finalChildrenVectors);
+ }
+
+ if (rowPosition.getIsNull() != null) {
+ setFieldNullFlag(rowPosition.getIsNull(), heapRowVector);
+ }
+ return Tuple2.of(levelDelegation, heapRowVector);
+ }
+
+ private Tuple2 readMap(
+ ParquetGroupField field, int readNumber, ColumnVector vector, boolean inside)
+ throws IOException {
+ HeapMapColumnVector mapVector = (HeapMapColumnVector) vector;
+ mapVector.reset();
+ List children = field.getChildren();
+ Preconditions.checkArgument(
+ children.size() == 2,
+ "Maps must have two type parameters, found %s",
+ children.size());
+ Tuple2 keyTuple =
+ readData(children.get(0), readNumber, mapVector.getKeyColumnVector(), true);
+ Tuple2 valueTuple =
+ readData(children.get(1), readNumber, mapVector.getValueColumnVector(), true);
+
+ LevelDelegation levelDelegation = keyTuple.f0;
+
+ CollectionPosition collectionPosition =
+ NestedPositionUtil.calculateCollectionOffsets(
+ field,
+ levelDelegation.getDefinitionLevel(),
+ levelDelegation.getRepetitionLevel());
+
+ // If map was inside the structure, then we need to renew the vector to reset the
+ // capacity.
+ if (inside) {
+ mapVector = new HeapMapColumnVector(collectionPosition.getValueCount(), keyTuple.f1, valueTuple.f1);
+ } else {
+ mapVector.setKeys(keyTuple.f1);
+ mapVector.setValues(valueTuple.f1);
+ }
+
+ if (collectionPosition.getIsNull() != null) {
+ setFieldNullFlag(collectionPosition.getIsNull(), mapVector);
+ }
+
+ mapVector.setLengths(collectionPosition.getLength());
+ mapVector.setOffsets(collectionPosition.getOffsets());
+
+ return Tuple2.of(levelDelegation, mapVector);
+ }
+
+ private Tuple2 readArray(
+ ParquetGroupField field, int readNumber, ColumnVector vector, boolean inside)
+ throws IOException {
+ HeapArrayVector arrayVector = (HeapArrayVector) vector;
+ arrayVector.reset();
+ List children = field.getChildren();
+ Preconditions.checkArgument(
+ children.size() == 1,
+ "Arrays must have a single type parameter, found %s",
+ children.size());
+ Tuple2 tuple =
+ readData(children.get(0), readNumber, arrayVector.getChild(), true);
+
+ LevelDelegation levelDelegation = tuple.f0;
+ CollectionPosition collectionPosition =
+ NestedPositionUtil.calculateCollectionOffsets(
+ field,
+ levelDelegation.getDefinitionLevel(),
+ levelDelegation.getRepetitionLevel());
+
+ // If array was inside the structure, then we need to renew the vector to reset the
+ // capacity.
+ if (inside) {
+ arrayVector = new HeapArrayVector(collectionPosition.getValueCount(), tuple.f1);
+ } else {
+ arrayVector.setChild(tuple.f1);
+ }
+
+ if (collectionPosition.getIsNull() != null) {
+ setFieldNullFlag(collectionPosition.getIsNull(), arrayVector);
+ }
+ arrayVector.setLengths(collectionPosition.getLength());
+ arrayVector.setOffsets(collectionPosition.getOffsets());
+ return Tuple2.of(levelDelegation, arrayVector);
+ }
+
+ private Tuple2 readPrimitive(
+ ParquetPrimitiveField field, int readNumber, ColumnVector vector) throws IOException {
+ ColumnDescriptor descriptor = field.getDescriptor();
+ NestedPrimitiveColumnReader reader = columnReaders.get(descriptor);
+ if (reader == null) {
+ reader =
+ new NestedPrimitiveColumnReader(
+ descriptor,
+ pages.getPageReader(descriptor),
+ isUtcTimestamp,
+ descriptor.getPrimitiveType(),
+ field.getType());
+ columnReaders.put(descriptor, reader);
+ }
+ WritableColumnVector writableColumnVector =
+ reader.readAndNewVector(readNumber, (WritableColumnVector) vector);
+ return Tuple2.of(reader.getLevelDelegation(), writableColumnVector);
+ }
+
+ private static void setFieldNullFlag(boolean[] nullFlags, AbstractHeapVector vector) {
+ for (int index = 0; index < vector.getLen() && index < nullFlags.length; index++) {
+ if (nullFlags[index]) {
+ vector.setNullAt(index);
+ }
+ }
+ }
+}
diff --git a/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/NestedPrimitiveColumnReader.java b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/NestedPrimitiveColumnReader.java
new file mode 100644
index 0000000000000..3f0aefe2af74f
--- /dev/null
+++ b/hudi-flink-datasource/hudi-flink1.18.x/src/main/java/org/apache/hudi/table/format/cow/vector/reader/NestedPrimitiveColumnReader.java
@@ -0,0 +1,639 @@
+/*
+ * 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.hudi.table.format.cow.vector.reader;
+
+import org.apache.hudi.table.format.cow.utils.IntArrayList;
+import org.apache.hudi.table.format.cow.vector.ParquetDecimalVector;
+import org.apache.hudi.table.format.cow.vector.position.LevelDelegation;
+
+import org.apache.flink.formats.parquet.vector.reader.ColumnReader;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBooleanVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapByteVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapBytesVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapDoubleVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapFloatVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapIntVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapLongVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapShortVector;
+import org.apache.flink.table.data.columnar.vector.heap.HeapTimestampVector;
+import org.apache.flink.table.data.columnar.vector.writable.WritableColumnVector;
+import org.apache.flink.table.types.logical.LogicalType;
+
+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.Encoding;
+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.page.DictionaryPage;
+import org.apache.parquet.column.page.PageReader;
+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.schema.PrimitiveType;
+import org.apache.parquet.schema.Type;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL;
+import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
+import static org.apache.parquet.column.ValuesType.VALUES;
+
+/**
+ * Reader to read a single primitive leaf column that participates in a nested (Dremel) structure.
+ *
+ *
Vendored from Apache Flink 2.1 (FLINK-35702, {@code
+ * org.apache.flink.formats.parquet.vector.reader.NestedPrimitiveColumnReader}). Only the package
+ * and the Hudi-local {@link ParquetDecimalVector} / {@link LevelDelegation} / {@link IntArrayList}
+ * imports are changed; the algorithm is untouched. The companion Hudi-specific {@code
+ * Int64TimestampColumnReader} / {@code FixedLenBytesColumnReader} behaviours stay at the leaf-
+ * reader creation boundary in {@code ParquetSplitReaderUtil} (lands in a follow-up PR), not inside
+ * this class — keeping it a faithful copy of upstream.
+ */
+public class NestedPrimitiveColumnReader implements ColumnReader {
+ private static final Logger LOG = LoggerFactory.getLogger(NestedPrimitiveColumnReader.class);
+
+ private final IntArrayList repetitionLevelList = new IntArrayList(0);
+ private final IntArrayList definitionLevelList = new IntArrayList(0);
+
+ private final PageReader pageReader;
+ private final ColumnDescriptor descriptor;
+ private final Type type;
+ private final LogicalType logicalType;
+
+ /** The dictionary, if this column has dictionary encoding. */
+ private final ParquetDataColumnReader dictionary;
+
+ /** Maximum definition level for this column. */
+ private final int maxDefLevel;
+
+ private boolean isUtcTimestamp;
+
+ /** Total number of values read. */
+ private long valuesRead;
+
+ /**
+ * value that indicates the end of the current page. That is, if valuesRead ==
+ * endOfPageValueCount, we are at the end of the page.
+ */
+ private long endOfPageValueCount;
+
+ /** If true, the current page is dictionary encoded. */
+ private boolean isCurrentPageDictionaryEncoded;
+
+ private int definitionLevel;
+ private int repetitionLevel;
+
+ /** Repetition/Definition/Value readers. */
+ private IntIterator repetitionLevelColumn;
+
+ private IntIterator definitionLevelColumn;
+ private ParquetDataColumnReader dataColumn;
+
+ /** Total values in the current page. */
+ private int pageValueCount;
+
+ // flag to indicate if there is no data in parquet data page
+ private boolean eof = false;
+
+ private boolean isFirstRow = true;
+
+ private Object lastValue;
+
+ public NestedPrimitiveColumnReader(
+ ColumnDescriptor descriptor,
+ PageReader pageReader,
+ boolean isUtcTimestamp,
+ Type parquetType,
+ LogicalType logicalType)
+ throws IOException {
+ this.descriptor = descriptor;
+ this.type = parquetType;
+ this.pageReader = pageReader;
+ this.maxDefLevel = descriptor.getMaxDefinitionLevel();
+ this.isUtcTimestamp = isUtcTimestamp;
+ this.logicalType = logicalType;
+
+ DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
+ if (dictionaryPage != null) {
+ try {
+ this.dictionary =
+ ParquetDataColumnReaderFactory.getDataColumnReaderByTypeOnDictionary(
+ parquetType.asPrimitiveType(),
+ dictionaryPage.getEncoding().initDictionary(descriptor, dictionaryPage),
+ isUtcTimestamp);
+ this.isCurrentPageDictionaryEncoded = true;
+ } catch (IOException e) {
+ throw new IOException(
+ String.format("Could not decode the dictionary for %s", descriptor), e);
+ }
+ } else {
+ this.dictionary = null;
+ this.isCurrentPageDictionaryEncoded = false;
+ }
+ }
+
+ // Not invoked directly; callers use readAndNewVector instead.
+ @Override
+ public void readToVector(int readNumber, WritableColumnVector vector) throws IOException {
+ throw new UnsupportedOperationException("This function should not be called.");
+ }
+
+ public WritableColumnVector readAndNewVector(int readNumber, WritableColumnVector vector)
+ throws IOException {
+ if (isFirstRow) {
+ if (!readValue()) {
+ return vector;
+ }
+ isFirstRow = false;
+ }
+
+ // index to set value.
+ int index = 0;
+ int valueIndex = 0;
+ List