gemini-code-assist[bot] commented on code in PR #38706: URL: https://github.com/apache/beam/pull/38706#discussion_r3309217957
########## sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/DeltaSourceDoFn.java: ########## @@ -0,0 +1,468 @@ +/* + * 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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 @Nullable Engine engine; + + private static final Logger LOG = LoggerFactory.getLogger(DeltaSourceDoFn.class); + + public DeltaSourceDoFn(@Nullable Map<String, String> hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } + + private Configuration getConfiguration() { + Configuration conf = new Configuration(); + if (hadoopConfig != null) { + for (Map.Entry<String, String> entry : hadoopConfig.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + } + return conf; + } Review Comment:  ### Performance Optimization: Cache Hadoop Configuration Creating a `new Configuration()` in Hadoop is a well-known performance bottleneck because it performs classpath scanning and XML parsing of default configuration files. Since `getConfiguration()` is called multiple times per task/bundle, caching the `Configuration` instance in a transient field of the `DoFn` will significantly reduce CPU overhead. ```java private transient @Nullable Configuration conf; private Configuration getConfiguration() { if (conf == null) { Configuration localConf = new Configuration(); if (hadoopConfig != null) { for (Map.Entry<String, String> entry : hadoopConfig.entrySet()) { localConf.set(entry.getKey(), entry.getValue()); } } conf = localConf; } return conf; } ``` ########## sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/BeamParquetHandler.java: ########## @@ -0,0 +1,367 @@ +/* + * 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; + + while (fileIter.hasNext()) { + 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; + } + + if (currentRgIndex >= tracker.currentRestriction().getTo()) { + // Skipping all blocks for the current file since they are located after the + // end index of the tracker. + currentRgIndex += fileBlocks; + continue; + } + + results.add( + readParquetFileDirect( + fileStatus, + hadoopPath, + metadata, + physicalSchema, + hasRowIndexCol, + currentRgIndex, + fileBlocks)); + + currentRgIndex += fileBlocks; + } Review Comment:  ### Performance & Resource Safety Improvement 1. **Avoid Unnecessary Footer Reads**: Currently, the loop reads the footer of all remaining files in `fileIter` even if `currentRgIndex` has already exceeded the tracker's restriction end index (`getTo()`). Since `currentRgIndex` is monotonically increasing, we can check `currentRgIndex >= tracker.currentRestriction().getTo()` **before** reading the footer and `break` the loop immediately. This avoids extremely expensive network I/O operations for files that are completely out of bounds. 2. **Resource Safety**: `fileIter` is a `CloseableIterator` and must be closed to prevent resource leaks. Wrapping the loop in a `try-finally` block ensures `fileIter.close()` is always called. ```java try { while (fileIter.hasNext()) { if (currentRgIndex >= tracker.currentRestriction().getTo()) { 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(); } ``` ########## sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/DeltaSourceDoFn.java: ########## @@ -0,0 +1,468 @@ +/* + * 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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 @Nullable Engine engine; + + private static final Logger LOG = LoggerFactory.getLogger(DeltaSourceDoFn.class); + + public DeltaSourceDoFn(@Nullable Map<String, String> hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } + + private Configuration getConfiguration() { + Configuration conf = new Configuration(); + if (hadoopConfig != null) { + for (Map.Entry<String, String> entry : hadoopConfig.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + } + return conf; + } + + // Returns the sizes of the row groups for a given DeltaReadTask. + private List<Long> getRowGroupSizes(DeltaReadTask task) { + List<Long> sizes = 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( + conf, + hadoopPath, + org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER); + for (org.apache.parquet.hadoop.metadata.BlockMetaData block : metadata.getBlocks()) { + sizes.add(block.getTotalByteSize()); + } + } catch (java.io.IOException e) { + throw new RuntimeException("Failed to read Parquet footer for " + pathStr, e); + } + } + return sizes; + } Review Comment:  ### Performance Optimization: Cache Task Metadata Currently, the Parquet footers of all files in a `DeltaReadTask` are read once in `newTracker` (via `getRowGroupSizes`) and then read again in `processElement` to get the block counts. Since `newTracker` and `processElement` are called sequentially on the same worker for the same element, we can cache the row group sizes and block counts of the most recently processed task in transient fields of the `DoFn`. This completely eliminates duplicate footer reads on the worker. ```java private transient @Nullable DeltaReadTask lastTask; private transient @Nullable List<Long> lastRowGroupSizes; private transient @Nullable List<Integer> lastFileBlockCounts; private void populateTaskMetadata(DeltaReadTask task) { if (task.equals(lastTask) && lastRowGroupSizes != null && lastFileBlockCounts != null) { return; } List<Long> sizes = new ArrayList<>(); List<Integer> 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( conf, hadoopPath, org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER); int blocks = metadata.getBlocks().size(); blockCounts.add(blocks); for (org.apache.parquet.hadoop.metadata.BlockMetaData block : metadata.getBlocks()) { sizes.add(block.getTotalByteSize()); } } catch (java.io.IOException e) { throw new RuntimeException("Failed to read Parquet footer for " + pathStr, e); } } lastTask = task; lastRowGroupSizes = sizes; lastFileBlockCounts = blockCounts; } private List<Long> getRowGroupSizes(DeltaReadTask task) { populateTaskMetadata(task); return lastRowGroupSizes; } private List<Integer> getFileBlockCounts(DeltaReadTask task) { populateTaskMetadata(task); return lastFileBlockCounts; } ``` ########## sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/DeltaSourceDoFn.java: ########## @@ -0,0 +1,468 @@ +/* + * 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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 @Nullable Engine engine; + + private static final Logger LOG = LoggerFactory.getLogger(DeltaSourceDoFn.class); + + public DeltaSourceDoFn(@Nullable Map<String, String> hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } + + private Configuration getConfiguration() { + Configuration conf = new Configuration(); + if (hadoopConfig != null) { + for (Map.Entry<String, String> entry : hadoopConfig.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + } + return conf; + } + + // Returns the sizes of the row groups for a given DeltaReadTask. + private List<Long> getRowGroupSizes(DeltaReadTask task) { + List<Long> sizes = 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( + conf, + hadoopPath, + org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER); + for (org.apache.parquet.hadoop.metadata.BlockMetaData block : metadata.getBlocks()) { + sizes.add(block.getTotalByteSize()); + } + } catch (java.io.IOException e) { + throw new RuntimeException("Failed to read Parquet footer for " + pathStr, e); + } + } + return sizes; + } + + @GetInitialRestriction + public OffsetRange getInitialRestriction(@Element DeltaReadTask task) { + List<Long> rowGroupSizes = getRowGroupSizes(task); + // Note that we use the number of row groups, `rowGroupSizes.size()`, here + // as the upper bound not the byte size of row groups. + return new OffsetRange(0L, rowGroupSizes.size()); + } + + @NewTracker + public DeltaReadTaskTracker newTracker( + @Restriction OffsetRange restriction, @Element DeltaReadTask task) { + return new DeltaReadTaskTracker(restriction, getRowGroupSizes(task)); + } + + @Setup + public void setUp() { + engine = DefaultEngine.create(getConfiguration()); + } + + @ProcessElement + public ProcessContinuation processElement( + @Element DeltaReadTask task, + RestrictionTracker<OffsetRange, Long> tracker, + OutputReceiver<Row> out) + throws Exception { + + SerializableRow scanStateRow = task.getScanStateRow(); + StructType physicalSchema = ScanStateRow.getPhysicalDataReadSchema(scanStateRow); + StructType logicalSchema = ScanStateRow.getLogicalSchema(scanStateRow); + Schema beamSchema = DeltaIO.ReadRows.convertToBeamSchema(logicalSchema); + + Engine currentEngine = engine; + if (currentEngine == null) { + throw new IllegalArgumentException("Expected the engine to not be null"); + } + + // `BeamParquetHandler` takes a reference to the `RestrictionTracker` so that it + // can perform `getFrom`, `getTo`, `tryClaim` requests to return the correct set + // of row groups that map to the current restriction. + BeamParquetHandler parquetHandler = + new BeamParquetHandler(getConfiguration(), currentEngine.getParquetHandler(), tracker); + BeamEngine beamEngine = new BeamEngine(currentEngine, parquetHandler); + + long currentStartRgIndex = 0L; + + // We have to go through files in the `DeltaReadTask` in order so that the + // `RestrictionTracker` + // can correctly handle the range of the current split. + for (SerializableRow scanFileRow : task.getScanFileRows()) { + FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRow); + + org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(fileStatus.getPath()); + org.apache.parquet.hadoop.metadata.ParquetMetadata metadata = + org.apache.parquet.hadoop.ParquetFileReader.readFooter( + getConfiguration(), + hadoopPath, + org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER); + long fileBlocks = metadata.getBlocks().size(); Review Comment:  ### Performance Optimization: Use Cached Block Counts Use the cached block counts instead of reading the Parquet footer of each file again during `processElement`. ```suggestion List<Integer> blockCounts = getFileBlockCounts(task); int fileIdx = 0; for (SerializableRow scanFileRow : task.getScanFileRows()) { FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRow); long fileBlocks = blockCounts.get(fileIdx++); ``` ########## sdks/java/io/delta/src/main/java/org/apache/beam/sdk/io/delta/DeltaSourceDoFn.java: ########## @@ -0,0 +1,468 @@ +/* + * 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 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 @Nullable Engine engine; + + private static final Logger LOG = LoggerFactory.getLogger(DeltaSourceDoFn.class); + + public DeltaSourceDoFn(@Nullable Map<String, String> hadoopConfig) { + this.hadoopConfig = hadoopConfig; + } + + private Configuration getConfiguration() { + Configuration conf = new Configuration(); + if (hadoopConfig != null) { + for (Map.Entry<String, String> entry : hadoopConfig.entrySet()) { + conf.set(entry.getKey(), entry.getValue()); + } + } + return conf; + } + + // Returns the sizes of the row groups for a given DeltaReadTask. + private List<Long> getRowGroupSizes(DeltaReadTask task) { + List<Long> sizes = 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( + conf, + hadoopPath, + org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER); + for (org.apache.parquet.hadoop.metadata.BlockMetaData block : metadata.getBlocks()) { + sizes.add(block.getTotalByteSize()); + } + } catch (java.io.IOException e) { + throw new RuntimeException("Failed to read Parquet footer for " + pathStr, e); + } + } + return sizes; + } + + @GetInitialRestriction + public OffsetRange getInitialRestriction(@Element DeltaReadTask task) { + List<Long> rowGroupSizes = getRowGroupSizes(task); + // Note that we use the number of row groups, `rowGroupSizes.size()`, here + // as the upper bound not the byte size of row groups. + return new OffsetRange(0L, rowGroupSizes.size()); + } + + @NewTracker + public DeltaReadTaskTracker newTracker( + @Restriction OffsetRange restriction, @Element DeltaReadTask task) { + return new DeltaReadTaskTracker(restriction, getRowGroupSizes(task)); + } + + @Setup + public void setUp() { + engine = DefaultEngine.create(getConfiguration()); + } + + @ProcessElement + public ProcessContinuation processElement( + @Element DeltaReadTask task, + RestrictionTracker<OffsetRange, Long> tracker, + OutputReceiver<Row> out) + throws Exception { + + SerializableRow scanStateRow = task.getScanStateRow(); + StructType physicalSchema = ScanStateRow.getPhysicalDataReadSchema(scanStateRow); + StructType logicalSchema = ScanStateRow.getLogicalSchema(scanStateRow); + Schema beamSchema = DeltaIO.ReadRows.convertToBeamSchema(logicalSchema); + + Engine currentEngine = engine; + if (currentEngine == null) { + throw new IllegalArgumentException("Expected the engine to not be null"); + } + + // `BeamParquetHandler` takes a reference to the `RestrictionTracker` so that it + // can perform `getFrom`, `getTo`, `tryClaim` requests to return the correct set + // of row groups that map to the current restriction. + BeamParquetHandler parquetHandler = + new BeamParquetHandler(getConfiguration(), currentEngine.getParquetHandler(), tracker); + BeamEngine beamEngine = new BeamEngine(currentEngine, parquetHandler); + + long currentStartRgIndex = 0L; + + // We have to go through files in the `DeltaReadTask` in order so that the + // `RestrictionTracker` + // can correctly handle the range of the current split. + for (SerializableRow scanFileRow : task.getScanFileRows()) { + FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFileRow); + + org.apache.hadoop.fs.Path hadoopPath = new org.apache.hadoop.fs.Path(fileStatus.getPath()); + org.apache.parquet.hadoop.metadata.ParquetMetadata metadata = + org.apache.parquet.hadoop.ParquetFileReader.readFooter( + getConfiguration(), + hadoopPath, + org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER); + long fileBlocks = metadata.getBlocks().size(); + + CloseableIterator<FileReadResult> fileReadResults = + parquetHandler.readParquetFiles( + Utils.singletonCloseableIterator(fileStatus), + physicalSchema, + Optional.empty(), + currentStartRgIndex); + + // Get the correct set of physical data for the current file that are within the + // range for the current `RestrictionTracker`. + CloseableIterator<ColumnarBatch> physicalData = + new CloseableIterator<ColumnarBatch>() { + @Override + public void close() throws java.io.IOException { + fileReadResults.close(); + } + + @Override + public boolean hasNext() { + return fileReadResults.hasNext(); + } + + @Override + public ColumnarBatch next() { + return fileReadResults.next().getData(); + } + }; + + // Convert physical data to logical data. + try (CloseableIterator<FilteredColumnarBatch> logicalBatches = + Scan.transformPhysicalData(beamEngine, scanStateRow, scanFileRow, physicalData)) { + + while (logicalBatches.hasNext()) { + FilteredColumnarBatch batch = logicalBatches.next(); + try (CloseableIterator<io.delta.kernel.data.Row> logicalRows = batch.getRows()) { + while (logicalRows.hasNext()) { + io.delta.kernel.data.Row deltaRow = logicalRows.next(); + Row beamRow = toBeamRow(deltaRow, beamSchema); + out.output(beamRow); + } + } + } + } Review Comment:  ### Resource Safety: Wrap fileReadResults in try-with-resources If `Scan.transformPhysicalData` throws an exception during its invocation, `fileReadResults` will be leaked. Wrapping `fileReadResults` in a try-with-resources block guarantees that it is always closed properly. ```java try (CloseableIterator<FileReadResult> fileReadResults = parquetHandler.readParquetFiles( Utils.singletonCloseableIterator(fileStatus), physicalSchema, Optional.empty(), currentStartRgIndex)) { // Get the correct set of physical data for the current file that are within the // range for the current `RestrictionTracker`. CloseableIterator<ColumnarBatch> physicalData = new CloseableIterator<ColumnarBatch>() { @Override public void close() throws java.io.IOException { fileReadResults.close(); } @Override public boolean hasNext() { return fileReadResults.hasNext(); } @Override public ColumnarBatch next() { return fileReadResults.next().getData(); } }; // Convert physical data to logical data. try (CloseableIterator<FilteredColumnarBatch> logicalBatches = Scan.transformPhysicalData(beamEngine, scanStateRow, scanFileRow, physicalData)) { while (logicalBatches.hasNext()) { FilteredColumnarBatch batch = logicalBatches.next(); try (CloseableIterator<io.delta.kernel.data.Row> logicalRows = batch.getRows()) { while (logicalRows.hasNext()) { io.delta.kernel.data.Row deltaRow = logicalRows.next(); Row beamRow = toBeamRow(deltaRow, beamSchema); out.output(beamRow); } } } } } ``` -- 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]
