chamikaramj commented on code in PR #38706:
URL: https://github.com/apache/beam/pull/38706#discussion_r3344720037


##########
sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/SerializableRow.java:
##########
@@ -0,0 +1,534 @@
+/*
+ * 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.beam.sdk.io.delta;
+
+import io.delta.kernel.data.ArrayValue;
+import io.delta.kernel.data.ColumnVector;
+import io.delta.kernel.data.MapValue;
+import io.delta.kernel.data.Row;
+import io.delta.kernel.types.ArrayType;
+import io.delta.kernel.types.BinaryType;
+import io.delta.kernel.types.BooleanType;
+import io.delta.kernel.types.ByteType;
+import io.delta.kernel.types.DataType;
+import io.delta.kernel.types.DecimalType;
+import io.delta.kernel.types.DoubleType;
+import io.delta.kernel.types.FloatType;
+import io.delta.kernel.types.IntegerType;
+import io.delta.kernel.types.LongType;
+import io.delta.kernel.types.MapType;
+import io.delta.kernel.types.ShortType;
+import io.delta.kernel.types.StringType;
+import io.delta.kernel.types.StructType;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A serializable wrapper for Delta {@link Row} that implements the {@link 
Row} interface itself,
+ * allowing worker nodes to access serialized Row objects using standard Delta 
Kernel APIs.
+ */
+public class SerializableRow implements Row, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final SerializableStructType schema;
+  private final @Nullable Object[] values;
+
+  public SerializableRow(Row row) {
+    this.schema = new SerializableStructType(row.getSchema());
+    StructType structType = row.getSchema();
+    int numFields = structType.fields().size();
+    this.values = new Object[numFields];
+    for (int i = 0; i < numFields; i++) {
+      DataType type = structType.fields().get(i).getDataType();
+      this.values[i] = getValue(row, i, type);
+    }
+  }
+
+  @Override
+  public StructType getSchema() {
+    return schema.get();
+  }
+
+  @Override
+  public boolean isNullAt(int ord) {
+    return values == null || values[ord] == null;
+  }
+
+  @Override
+  public boolean getBoolean(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Boolean) val : false;
+  }
+
+  @Override
+  public byte getByte(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Byte) val : 0;
+  }
+
+  @Override
+  public short getShort(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Short) val : 0;
+  }
+
+  @Override
+  public int getInt(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Integer) val : 0;
+  }
+
+  @Override
+  public long getLong(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Long) val : 0L;
+  }
+
+  @Override
+  public float getFloat(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Float) val : 0.0f;
+  }
+
+  @Override
+  public double getDouble(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Double) val : 0.0d;
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public String getString(int ord) {
+    return (String) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public byte[] getBinary(int ord) {
+    return (byte[]) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public BigDecimal getDecimal(int ord) {
+    return (BigDecimal) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public Row getStruct(int ord) {
+    return (Row) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings({"unchecked", "nullness"})
+  public ArrayValue getArray(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    if (val == null) {
+      return null;
+    }
+    DataType elementType =
+        ((ArrayType) 
getSchema().fields().get(ord).getDataType()).getElementType();
+    return new SerializableArrayValue((List<@Nullable Object>) val, 
elementType);
+  }
+
+  @Override
+  @SuppressWarnings({"unchecked", "nullness"})
+  public MapValue getMap(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    if (val == null) {
+      return null;
+    }
+    MapType mapType = (MapType) getSchema().fields().get(ord).getDataType();
+    return new SerializableMapValue(
+        (Map<Object, @Nullable Object>) val, mapType.getKeyType(), 
mapType.getValueType());
+  }
+
+  @Override
+  public boolean equals(@Nullable Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof SerializableRow)) {
+      return false;
+    }
+    SerializableRow that = (SerializableRow) o;
+    return Objects.equals(schema, that.schema) && 
java.util.Arrays.deepEquals(values, that.values);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(schema, java.util.Arrays.deepHashCode(values));
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("SerializableRow{schema=").append(schema).append(", values=[");
+    if (values != null) {
+      for (int i = 0; i < values.length; i++) {
+        if (i > 0) {
+          sb.append(", ");
+        }
+        sb.append(values[i]);
+      }
+    }
+    sb.append("]}");
+    return sb.toString();
+  }
+
+  private static @Nullable Object getValue(Row row, int index, DataType type) {
+    if (row.isNullAt(index)) {
+      return null;
+    }
+    if (type instanceof BooleanType) {
+      return row.getBoolean(index);
+    } else if (type instanceof ByteType) {
+      return row.getByte(index);
+    } else if (type instanceof ShortType) {
+      return row.getShort(index);
+    } else if (type instanceof IntegerType) {
+      return row.getInt(index);
+    } else if (type instanceof LongType) {
+      return row.getLong(index);
+    } else if (type instanceof FloatType) {
+      return row.getFloat(index);
+    } else if (type instanceof DoubleType) {
+      return row.getDouble(index);
+    } else if (type instanceof StringType) {
+      return row.getString(index);
+    } else if (type instanceof BinaryType) {
+      return row.getBinary(index);
+    } else if (type instanceof DecimalType) {
+      return row.getDecimal(index);
+    } else if (type instanceof StructType) {
+      return new SerializableRow(row.getStruct(index));
+    } else if (type instanceof ArrayType) {
+      ArrayValue arr = row.getArray(index);
+      return convertArray(arr, (ArrayType) type);
+    } else if (type instanceof MapType) {
+      MapValue map = row.getMap(index);
+      return convertMap(map, (MapType) type);
+    }
+    throw new IllegalArgumentException("Unsupported type: " + type);

Review Comment:
   Done.



##########
sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/SerializableRow.java:
##########
@@ -0,0 +1,534 @@
+/*
+ * 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.beam.sdk.io.delta;
+
+import io.delta.kernel.data.ArrayValue;
+import io.delta.kernel.data.ColumnVector;
+import io.delta.kernel.data.MapValue;
+import io.delta.kernel.data.Row;
+import io.delta.kernel.types.ArrayType;
+import io.delta.kernel.types.BinaryType;
+import io.delta.kernel.types.BooleanType;
+import io.delta.kernel.types.ByteType;
+import io.delta.kernel.types.DataType;
+import io.delta.kernel.types.DecimalType;
+import io.delta.kernel.types.DoubleType;
+import io.delta.kernel.types.FloatType;
+import io.delta.kernel.types.IntegerType;
+import io.delta.kernel.types.LongType;
+import io.delta.kernel.types.MapType;
+import io.delta.kernel.types.ShortType;
+import io.delta.kernel.types.StringType;
+import io.delta.kernel.types.StructType;
+import java.io.Serializable;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A serializable wrapper for Delta {@link Row} that implements the {@link 
Row} interface itself,
+ * allowing worker nodes to access serialized Row objects using standard Delta 
Kernel APIs.
+ */
+public class SerializableRow implements Row, Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final SerializableStructType schema;
+  private final @Nullable Object[] values;
+
+  public SerializableRow(Row row) {
+    this.schema = new SerializableStructType(row.getSchema());
+    StructType structType = row.getSchema();
+    int numFields = structType.fields().size();
+    this.values = new Object[numFields];
+    for (int i = 0; i < numFields; i++) {
+      DataType type = structType.fields().get(i).getDataType();
+      this.values[i] = getValue(row, i, type);
+    }
+  }
+
+  @Override
+  public StructType getSchema() {
+    return schema.get();
+  }
+
+  @Override
+  public boolean isNullAt(int ord) {
+    return values == null || values[ord] == null;
+  }
+
+  @Override
+  public boolean getBoolean(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Boolean) val : false;
+  }
+
+  @Override
+  public byte getByte(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Byte) val : 0;
+  }
+
+  @Override
+  public short getShort(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Short) val : 0;
+  }
+
+  @Override
+  public int getInt(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Integer) val : 0;
+  }
+
+  @Override
+  public long getLong(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Long) val : 0L;
+  }
+
+  @Override
+  public float getFloat(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Float) val : 0.0f;
+  }
+
+  @Override
+  public double getDouble(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    return val != null ? (Double) val : 0.0d;
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public String getString(int ord) {
+    return (String) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public byte[] getBinary(int ord) {
+    return (byte[]) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public BigDecimal getDecimal(int ord) {
+    return (BigDecimal) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings("nullness")
+  public Row getStruct(int ord) {
+    return (Row) Objects.requireNonNull(values)[ord];
+  }
+
+  @Override
+  @SuppressWarnings({"unchecked", "nullness"})
+  public ArrayValue getArray(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    if (val == null) {
+      return null;
+    }
+    DataType elementType =
+        ((ArrayType) 
getSchema().fields().get(ord).getDataType()).getElementType();
+    return new SerializableArrayValue((List<@Nullable Object>) val, 
elementType);
+  }
+
+  @Override
+  @SuppressWarnings({"unchecked", "nullness"})
+  public MapValue getMap(int ord) {
+    Object val = Objects.requireNonNull(values)[ord];
+    if (val == null) {
+      return null;
+    }
+    MapType mapType = (MapType) getSchema().fields().get(ord).getDataType();
+    return new SerializableMapValue(
+        (Map<Object, @Nullable Object>) val, mapType.getKeyType(), 
mapType.getValueType());
+  }
+
+  @Override
+  public boolean equals(@Nullable Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (!(o instanceof SerializableRow)) {
+      return false;
+    }
+    SerializableRow that = (SerializableRow) o;
+    return Objects.equals(schema, that.schema) && 
java.util.Arrays.deepEquals(values, that.values);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(schema, java.util.Arrays.deepHashCode(values));
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("SerializableRow{schema=").append(schema).append(", values=[");
+    if (values != null) {
+      for (int i = 0; i < values.length; i++) {
+        if (i > 0) {
+          sb.append(", ");
+        }
+        sb.append(values[i]);
+      }
+    }
+    sb.append("]}");
+    return sb.toString();
+  }
+
+  private static @Nullable Object getValue(Row row, int index, DataType type) {
+    if (row.isNullAt(index)) {
+      return null;
+    }
+    if (type instanceof BooleanType) {
+      return row.getBoolean(index);
+    } else if (type instanceof ByteType) {
+      return row.getByte(index);
+    } else if (type instanceof ShortType) {
+      return row.getShort(index);
+    } else if (type instanceof IntegerType) {
+      return row.getInt(index);
+    } else if (type instanceof LongType) {
+      return row.getLong(index);
+    } else if (type instanceof FloatType) {
+      return row.getFloat(index);
+    } else if (type instanceof DoubleType) {
+      return row.getDouble(index);
+    } else if (type instanceof StringType) {
+      return row.getString(index);
+    } else if (type instanceof BinaryType) {
+      return row.getBinary(index);
+    } else if (type instanceof DecimalType) {
+      return row.getDecimal(index);
+    } else if (type instanceof StructType) {
+      return new SerializableRow(row.getStruct(index));
+    } else if (type instanceof ArrayType) {
+      ArrayValue arr = row.getArray(index);
+      return convertArray(arr, (ArrayType) type);
+    } else if (type instanceof MapType) {
+      MapValue map = row.getMap(index);
+      return convertMap(map, (MapType) type);
+    }
+    throw new IllegalArgumentException("Unsupported type: " + type);
+  }
+
+  private static @Nullable Object getVectorValue(ColumnVector vector, int 
index, DataType type) {
+    if (vector.isNullAt(index)) {
+      return null;
+    }
+    if (type instanceof BooleanType) {
+      return vector.getBoolean(index);
+    } else if (type instanceof ByteType) {
+      return vector.getByte(index);
+    } else if (type instanceof ShortType) {
+      return vector.getShort(index);
+    } else if (type instanceof IntegerType) {
+      return vector.getInt(index);
+    } else if (type instanceof LongType) {
+      return vector.getLong(index);
+    } else if (type instanceof FloatType) {
+      return vector.getFloat(index);
+    } else if (type instanceof DoubleType) {
+      return vector.getDouble(index);
+    } else if (type instanceof StringType) {
+      return vector.getString(index);
+    } else if (type instanceof BinaryType) {
+      return vector.getBinary(index);
+    } else if (type instanceof DecimalType) {
+      return vector.getDecimal(index);
+    } else if (type instanceof StructType) {
+      StructType structType = (StructType) type;
+      int numFields = structType.fields().size();
+      ColumnVector[] childFields = new ColumnVector[numFields];
+      for (int j = 0; j < numFields; j++) {
+        childFields[j] = vector.getChild(j);
+      }
+      return new SerializableRow(new VectorRow(structType, childFields, 
index));
+    } else if (type instanceof ArrayType) {
+      ArrayValue arr = vector.getArray(index);
+      return convertArray(arr, (ArrayType) type);
+    } else if (type instanceof MapType) {
+      MapValue map = vector.getMap(index);
+      return convertMap(map, (MapType) type);
+    }
+    throw new IllegalArgumentException("Unsupported vector type: " + type);

Review Comment:
   Done.



##########
sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/BeamParquetHandler.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.beam.sdk.io.delta;
+
+import io.delta.kernel.data.FilteredColumnarBatch;
+import 
io.delta.kernel.defaults.internal.parquet.ParquetFileReader.BatchReadSupport;
+import io.delta.kernel.engine.FileReadResult;
+import io.delta.kernel.engine.ParquetHandler;
+import io.delta.kernel.expressions.Column;
+import io.delta.kernel.expressions.Predicate;
+import io.delta.kernel.types.MetadataColumnSpec;
+import io.delta.kernel.types.StructType;
+import io.delta.kernel.utils.CloseableIterator;
+import io.delta.kernel.utils.DataFileStatus;
+import io.delta.kernel.utils.FileStatus;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.NoSuchElementException;
+import java.util.Optional;
+import java.util.Set;
+import org.apache.beam.sdk.io.range.OffsetRange;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.format.converter.ParquetMetadataConverter;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.api.InitContext;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.metadata.FileMetaData;
+import org.apache.parquet.hadoop.metadata.ParquetMetadata;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.ColumnIOFactory;
+import org.apache.parquet.io.MessageColumnIO;
+import org.apache.parquet.io.RecordReader;
+import org.apache.parquet.io.api.RecordMaterializer;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * A Beam specific {@link ParquetHandler} that delegates row group claiming to 
a {@link
+ * DeltaReadTaskTracker}.
+ */
+public class BeamParquetHandler implements ParquetHandler {
+  private final Configuration conf;
+  private final ParquetHandler delegate;
+  private final RestrictionTracker<OffsetRange, Long> tracker;
+  private static final long DEFAULT_START_RG_INDEX = 0L;
+
+  public BeamParquetHandler(
+      Configuration conf, ParquetHandler delegate, 
RestrictionTracker<OffsetRange, Long> tracker) {
+    this.conf = conf;
+    this.delegate = delegate;
+    this.tracker = tracker;
+  }
+
+  private boolean claimFailed = false;
+
+  /**
+   * A method that is expected to be called after the first file processing is 
done. It returns
+   * whether the last file process resulted in a claim failure. This allows 
the caller to skip
+   * trying to read the remaining files of the task which would result in 
claim failures for each
+   * row group within them.
+   *
+   * @return true, if the last file process resulted in a claim failure. 
Returns false otherwise.
+   */
+  public boolean hasClaimFailed() {
+    return claimFailed;
+  }
+
+  @Override
+  public CloseableIterator<FileReadResult> readParquetFiles(
+      CloseableIterator<FileStatus> fileIter,
+      StructType physicalSchema,
+      Optional<Predicate> predicate)
+      throws IOException {
+    return readParquetFiles(fileIter, physicalSchema, predicate, 
DEFAULT_START_RG_INDEX);
+  }
+
+  /**
+   * Reads Parquet files starting from a given row group index.
+   *
+   * <p>This takes the {@code RestrictionTracker} referenced by the current 
{@code ParquetReader}
+   * into consideration when reading by performing the following.
+   *
+   * <p>* Skips blocks of the set of files till the given start row group 
index or the start point
+   * of the {@code RestrictionTracker}, whatever is higher. * Invokes {@code 
tryClaim} when reading
+   * a specific block stops reading if a {@code tryClaim} fails. * Stops 
reading if the end of the
+   * range of the {@code RestrictionTracker} is reached.
+   *
+   * <p>If {@code tryClaim} fails during reading, subsequent {@code 
hasClaimFailed} calls will
+   * return {@code true}, so the caller can skip reading subsequent files that 
are in the range
+   * being considered for reading.
+   */
+  public CloseableIterator<FileReadResult> readParquetFiles(
+      CloseableIterator<FileStatus> fileIter,
+      StructType physicalSchema,
+      Optional<Predicate> predicate,
+      long startRgIndex)
+      throws IOException {
+
+    List<CloseableIterator<FileReadResult>> results = new ArrayList<>();
+    boolean hasRowIndexCol = 
physicalSchema.contains(MetadataColumnSpec.ROW_INDEX);
+
+    long currentRgIndex = startRgIndex;
+
+    try {
+      while (fileIter.hasNext()) {
+        if (currentRgIndex >= tracker.currentRestriction().getTo()) {
+          // Skipping all blocks for the remaining files since they are 
located after the
+          // end index of the tracker. Since currentRgIndex is monotonically 
increasing,
+          // we can break the loop immediately to avoid extremely expensive 
network I/O.
+          break;
+        }
+
+        FileStatus fileStatus = fileIter.next();
+        Path hadoopPath = new Path(fileStatus.getPath());
+        ParquetMetadata metadata =
+            ParquetFileReader.readFooter(conf, hadoopPath, 
ParquetMetadataConverter.NO_FILTER);
+        long fileBlocks = metadata.getBlocks().size();
+
+        if (currentRgIndex + fileBlocks <= 
tracker.currentRestriction().getFrom()) {
+          // Skipping all blocks for the current file since they are located 
before the
+          // start index of the tracker.
+          currentRgIndex += fileBlocks;
+          continue;
+        }
+
+        results.add(
+            readParquetFileDirect(
+                fileStatus,
+                hadoopPath,
+                metadata,
+                physicalSchema,
+                hasRowIndexCol,
+                currentRgIndex,
+                fileBlocks));
+
+        currentRgIndex += fileBlocks;
+      }
+    } finally {
+      fileIter.close();
+    }
+
+    return combineResults(results);
+  }
+
+  // Reads the correct set of blocks that belong to the given Parquet file that
+  // are within range for the current `RestrictionTracker`. If the current file
+  // has some blocks that are within the tracker's range and some that are
+  // outside,
+  // this will only read the blocks that are within the range.
+  private CloseableIterator<FileReadResult> readParquetFileDirect(
+      FileStatus fileStatus,
+      Path hadoopPath,
+      ParquetMetadata metadata,
+      StructType physicalSchema,
+      boolean hasRowIndexCol,
+      long startRgIndex,
+      long fileBlocks) {
+
+    return new CloseableIterator<FileReadResult>() {
+      @javax.annotation.Nullable private ParquetFileReader reader = null;
+      @javax.annotation.Nullable private BatchReadSupport readSupport = null;
+      @javax.annotation.Nullable private RecordMaterializer<Object> 
recordConverter = null;
+      @javax.annotation.Nullable private MessageColumnIO columnIO = null;
+
+      private long currentRgOffset = 0;
+      @javax.annotation.Nullable private RecordReader<Object> 
currentRecordReader = null;
+      private long currentRgTotalRows = 0;
+      private long currentRgRowOffset = 0;
+      private long currentRgStartingRowIndex = 0;
+
+      @javax.annotation.Nullable private FileReadResult nextResult = null;
+      private boolean isDone = false;
+
+      private void initReaderIfRequired() throws IOException {
+        if (reader != null) {
+          return;
+        }
+        HadoopInputFile inputFile = HadoopInputFile.fromPath(hadoopPath, conf);
+        ParquetFileReader localReader = ParquetFileReader.open(inputFile);
+        reader = localReader;
+
+        FileMetaData fileMetaData = metadata.getFileMetaData();
+        MessageType fileSchema = fileMetaData.getSchema();
+        Map<String, Set<String>> keyValueMetadata = new HashMap<>();
+        if (fileMetaData.getKeyValueMetaData() != null) {
+          for (Map.Entry<String, String> entry : 
fileMetaData.getKeyValueMetaData().entrySet()) {
+            keyValueMetadata.put(entry.getKey(), 
Collections.singleton(entry.getValue()));
+          }
+        }
+
+        BatchReadSupport localReadSupport = new BatchReadSupport(1024, 
physicalSchema);
+        readSupport = localReadSupport;
+        ReadSupport.ReadContext readContext =
+            localReadSupport.init(new InitContext(conf, keyValueMetadata, 
fileSchema));
+        RecordMaterializer<Object> localRecordConverter =
+            localReadSupport.prepareForRead(
+                conf, fileMetaData.getKeyValueMetaData(), fileSchema, 
readContext);
+        recordConverter = localRecordConverter;
+        localReader.setRequestedSchema(readContext.getRequestedSchema());
+
+        ColumnIOFactory columnIOFactory = new 
ColumnIOFactory(fileMetaData.getCreatedBy());
+        columnIO = 
columnIOFactory.getColumnIO(readContext.getRequestedSchema(), fileSchema, true);
+      }
+
+      @Override
+      public boolean hasNext() {
+        if (isDone) {
+          return false;
+        }
+        if (nextResult != null) {
+          return true;
+        }
+
+        try {
+          initReaderIfRequired();
+          ParquetFileReader localReader = reader;
+          BatchReadSupport localReadSupport = readSupport;
+          MessageColumnIO localColumnIO = columnIO;
+          RecordMaterializer<Object> localRecordConverter = recordConverter;
+          if (localReader == null
+              || localReadSupport == null
+              || localColumnIO == null
+              || localRecordConverter == null) {
+            throw new IllegalStateException("Reader not initialized");
+          }
+
+          while (true) {
+            RecordReader<Object> localRecordReader = currentRecordReader;
+            if (localRecordReader != null && currentRgRowOffset < 
currentRgTotalRows) {
+              int batchSize = (int) Math.min(1024L, currentRgTotalRows - 
currentRgRowOffset);
+              for (int i = 0; i < batchSize; i++) {
+                localRecordReader.read();
+                long rowIndex =
+                    hasRowIndexCol ? (currentRgStartingRowIndex + 
currentRgRowOffset + i) : -1L;
+                localReadSupport.finalizeCurrentRow(rowIndex);
+              }
+              currentRgRowOffset += batchSize;
+              io.delta.kernel.data.ColumnarBatch batch =
+                  localReadSupport.getDataAsColumnarBatch(batchSize);
+              nextResult = new FileReadResult(batch, fileStatus.getPath());
+              return true;
+            }
+
+            currentRecordReader = null;
+            if (currentRgOffset >= fileBlocks) {
+              isDone = true;
+              return false;
+            }
+
+            // Checking the range for specific row groups.
+            long rgIndex = startRgIndex + currentRgOffset;
+            if (rgIndex < tracker.currentRestriction().getFrom()
+                || rgIndex >= tracker.currentRestriction().getTo()) {
+              localReader.skipNextRowGroup();

Review Comment:
   Done.



##########
sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/DeltaSourceDoFn.java:
##########
@@ -0,0 +1,498 @@
+/*
+ * 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.beam.sdk.io.delta;
+
+import io.delta.kernel.Scan;
+import io.delta.kernel.data.ArrayValue;
+import io.delta.kernel.data.ColumnVector;
+import io.delta.kernel.data.ColumnarBatch;
+import io.delta.kernel.data.FilteredColumnarBatch;
+import io.delta.kernel.data.MapValue;
+import io.delta.kernel.defaults.engine.DefaultEngine;
+import io.delta.kernel.engine.Engine;
+import io.delta.kernel.engine.FileReadResult;
+import io.delta.kernel.internal.InternalScanFileUtils;
+import io.delta.kernel.internal.data.ScanStateRow;
+import io.delta.kernel.internal.util.Utils;
+import io.delta.kernel.types.ArrayType;
+import io.delta.kernel.types.BinaryType;
+import io.delta.kernel.types.BooleanType;
+import io.delta.kernel.types.ByteType;
+import io.delta.kernel.types.DataType;
+import io.delta.kernel.types.DateType;
+import io.delta.kernel.types.DoubleType;
+import io.delta.kernel.types.FloatType;
+import io.delta.kernel.types.IntegerType;
+import io.delta.kernel.types.LongType;
+import io.delta.kernel.types.MapType;
+import io.delta.kernel.types.ShortType;
+import io.delta.kernel.types.StringType;
+import io.delta.kernel.types.StructField;
+import io.delta.kernel.types.StructType;
+import io.delta.kernel.types.TimestampType;
+import io.delta.kernel.utils.CloseableIterator;
+import io.delta.kernel.utils.FileStatus;
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.beam.sdk.io.range.OffsetRange;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.values.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A Splittable DoFn that processes {@link DeltaReadTask} elements, performs 
logical reads, and
+ * supports dynamic work rebalancing.
+ */
[email protected]
+class DeltaSourceDoFn extends DoFn<DeltaReadTask, Row> {
+  @Nullable Map<String, String> hadoopConfig;
+  private transient @Nullable Engine engine;
+  private transient @Nullable Configuration conf;
+
+  private transient @Nullable DeltaReadTask cachedTask;
+  private transient @Nullable List<Long> cachedRowGroupSizes;
+  private transient @Nullable List<Long> cachedBlockCountsPerFile;
+
+  public DeltaSourceDoFn(@Nullable Map<String, String> hadoopConfig) {
+    this.hadoopConfig = hadoopConfig;
+  }
+
+  private synchronized Configuration getConfiguration() {
+    Configuration localConf = conf;
+    if (localConf == null) {
+      localConf = new Configuration();
+      if (hadoopConfig != null) {
+        for (Map.Entry<String, String> entry : hadoopConfig.entrySet()) {
+          localConf.set(entry.getKey(), entry.getValue());
+        }
+      }
+      conf = localConf;
+    }
+    return localConf;
+  }
+
+  private synchronized @Nullable List<Long> getCachedBlockCounts(DeltaReadTask 
task) {
+    if (task.equals(cachedTask)) {
+      return cachedBlockCountsPerFile;
+    }
+    return null;
+  }
+
+  // Returns the sizes of the row groups for a given DeltaReadTask.
+  private synchronized List<Long> getRowGroupSizes(DeltaReadTask task) {
+    if (task.equals(cachedTask) && cachedRowGroupSizes != null) {
+      return cachedRowGroupSizes;
+    }
+
+    List<Long> sizes = new ArrayList<>();
+    List<Long> blockCounts = new ArrayList<>();
+    Configuration conf = getConfiguration();
+    for (SerializableRow scanFileRow : task.getScanFileRows()) {
+      String pathStr = 
InternalScanFileUtils.getAddFileStatus(scanFileRow).getPath();
+      try {
+        org.apache.hadoop.fs.Path hadoopPath = new 
org.apache.hadoop.fs.Path(pathStr);
+        org.apache.parquet.hadoop.metadata.ParquetMetadata metadata =
+            org.apache.parquet.hadoop.ParquetFileReader.readFooter(

Review Comment:
   Done.



##########
sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/DeltaReadTask.java:
##########
@@ -0,0 +1,76 @@
+/*

Review Comment:
   This specific file is just a builder so don't think needs a dedicate unit 
test. DeltaIOTest should cover the main flow including DoFns. Note that I added 
SerializableRowTest for testing serialization/deserialization specifically. 
Lemme know if you see any specific unit test gaps.



-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to