ahmedabu98 commented on code in PR #33504: URL: https://github.com/apache/beam/pull/33504#discussion_r1948358867
########## sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java: ########## @@ -0,0 +1,112 @@ +/* + * 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.iceberg; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutionException; +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.util.ShardedKey; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Unbounded read implementation. + * + * <p>An SDF that takes a batch of {@link ReadTask}s. For each task, reads Iceberg {@link Record}s, + * and converts to Beam {@link Row}s. + * + * <p>The split granularity is set to the incoming batch size, i.e. the number of potential splits Review Comment: Mentioning this in https://s.apache.org/beam-iceberg-incremental-source > I cannot recall - will there always be a shuffle upstream of this Yes, currently there is a GroupIntoBatches step which contains a shuffle. I found that a Reshuffle/Redistribute into processing individual ReadTasks will almost always lead to OOMs (and stuck pipeline) for a decent read size because the worker will try buffering all files concurrently. We could suggest setting a large number of workers to start with, but I think that's not a great user experience. They would have to experiment a bit to figure that one out. Grouping into batches is much easier to work with because we're not buffering all files at once. For Dataflow, GiB will also send signals to increase key fanout based on backlog, which helps with dynamic worker autoscaling. -- 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]
