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


##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/io/iceberg/ScanSource.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.io.iceberg;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.io.BoundedSource;
+import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.transforms.display.DisplayData;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.BaseCombinedScanTask;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.io.CloseableIterable;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+class ScanSource extends BoundedSource<Row> {
+
+  private @Nullable CombinedScanTask task;
+  private IcebergScanConfig scan;
+
+  public ScanSource(IcebergScanConfig scan, @Nullable CombinedScanTask task) {
+    this.task = task;
+    this.scan = scan;
+  }
+
+  public ScanSource(IcebergScanConfig scan) {
+    this(scan, null);
+  }
+
+  public Catalog catalog() {
+    return scan.getCatalogConfig().catalog();
+  }
+
+  public Table table() {
+    return 
catalog().loadTable(TableIdentifier.parse(scan.getTableIdentifier()));
+  }
+
+  private TableScan getTableScan() {
+    // Always project to our destination schema
+    return table()
+        .newScan()
+        
.project(SchemaAndRowConversions.beamSchemaToIcebergSchema(scan.getSchema()));
+  }
+
+  private CombinedScanTask getTask() {
+    if (task == null) {
+      task = new 
BaseCombinedScanTask(ImmutableList.copyOf(getTableScan().planFiles()));
+    }
+    return task;
+  }
+
+  @Override
+  public List<? extends BoundedSource<Row>> split(
+      long desiredBundleSizeBytes, PipelineOptions options) throws Exception {
+    ArrayList<ScanSource> splits = new ArrayList<>();
+    switch (scan.getScanType()) {
+      case TABLE:
+        TableScan tableScan = getTableScan();
+
+        // Override the split size with our desired size
+        if (desiredBundleSizeBytes > 0) {
+          tableScan =
+              tableScan.option(TableProperties.SPLIT_SIZE, 
Long.toString(desiredBundleSizeBytes));
+        }
+        if (scan.getFilter() != null) {
+          tableScan = tableScan.filter(scan.getFilter());
+        }
+        if (scan.getCaseSensitive() != null) {
+          tableScan = tableScan.caseSensitive(scan.getCaseSensitive());
+        }
+        if (scan.getSnapshot() != null) {
+          tableScan = tableScan.useSnapshot(scan.getSnapshot());
+        }
+        if (scan.getBranch() != null) {
+          tableScan = tableScan.useRef(scan.getBranch());
+        } else if (scan.getTag() != null) {
+          tableScan = tableScan.useRef(scan.getTag());
+        }
+        try (CloseableIterable<CombinedScanTask> t = tableScan.planTasks()) {
+          for (CombinedScanTask c : t) {
+            splits.add(new ScanSource(scan, c));
+          }
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+        break;
+      case BATCH:
+        throw new UnsupportedOperationException("BATCH scan not supported");

Review Comment:
   Also fail for the default path prevent dataloss (or just return the original 
source if that can be read directly).



-- 
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: github-unsubscr...@beam.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to