kennknowles commented on code in PR #33504:
URL: https://github.com/apache/beam/pull/33504#discussion_r1951250830


##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java:
##########
@@ -413,29 +438,56 @@ abstract static class Builder {
 
       abstract Builder setTableIdentifier(TableIdentifier identifier);
 
+      abstract Builder setFromSnapshotExclusive(@Nullable Long 
fromSnapshotExclusive);
+
+      abstract Builder setToSnapshot(@Nullable Long toSnapshot);
+
+      abstract Builder setTriggeringFrequency(Duration triggeringFrequency);
+
       abstract ReadRows build();
     }
 
     public ReadRows from(TableIdentifier tableIdentifier) {
       return toBuilder().setTableIdentifier(tableIdentifier).build();
     }
 
+    public ReadRows fromSnapshotExclusive(@Nullable Long 
fromSnapshotExclusive) {
+      return 
toBuilder().setFromSnapshotExclusive(fromSnapshotExclusive).build();
+    }
+
+    public ReadRows toSnapshot(@Nullable Long toSnapshot) {
+      return toBuilder().setToSnapshot(toSnapshot).build();
+    }
+
+    public ReadRows withTriggeringFrequency(Duration triggeringFrequency) {
+      return toBuilder().setTriggeringFrequency(triggeringFrequency).build();
+    }
+
     @Override
     public PCollection<Row> expand(PBegin input) {
       TableIdentifier tableId =
           checkStateNotNull(getTableIdentifier(), "Must set a table to read 
from.");
 
       Table table = getCatalogConfig().catalog().loadTable(tableId);
 
-      return input.apply(
-          Read.from(
-              new ScanSource(
-                  IcebergScanConfig.builder()
-                      .setCatalogConfig(getCatalogConfig())
-                      .setScanType(IcebergScanConfig.ScanType.TABLE)
-                      .setTableIdentifier(tableId)
-                      
.setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema()))
-                      .build())));
+      IcebergScanConfig scanConfig =
+          IcebergScanConfig.builder()
+              .setCatalogConfig(getCatalogConfig())
+              .setScanType(IcebergScanConfig.ScanType.TABLE)
+              .setTableIdentifier(tableId)
+              
.setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema()))
+              .setFromSnapshotExclusive(getFromSnapshotExclusive())
+              .setToSnapshot(getToSnapshot())
+              .build();
+      if (getTriggeringFrequency() != null

Review Comment:
   OK. I don't love it but this will be a pretty minor part of how a user 
interacts with it anyhow. Noting that the Flink source has explicit 
`streaming(true)` 😛 
   
   If we support for incremental reads including deletes for CDC we'll need it 
to be more explicit and have a different output type.



##########
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:
   OK this makes sense. Note that if you do a `Redistribute` in streaming 
without specifying a number of shards, you basically get one element per key 
aka one element bundles so that is why you would see no good splitting and 
extremely high concurrency.



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