chamikaramj commented on code in PR #38706: URL: https://github.com/apache/beam/pull/38706#discussion_r3309572904
########## 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: Done ########## 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: Done. -- 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]
