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


##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java:
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.catalog.Catalog;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** Utility to fetch and cache Iceberg {@link Table}s. */
+class TableCache {
+  private static final Cache<TableIdentifier, Table> CACHE =
+      CacheBuilder.newBuilder().expireAfterWrite(3, TimeUnit.MINUTES).build();

Review Comment:
   Done



##########
sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java:
##########
@@ -0,0 +1,191 @@
+/*
+ * 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 static org.apache.beam.sdk.transforms.Watch.Growth.PollResult;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.coders.ListCoder;
+import org.apache.beam.sdk.metrics.Counter;
+import org.apache.beam.sdk.metrics.Gauge;
+import org.apache.beam.sdk.metrics.Metrics;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.ValueState;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.Watch;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.TimestampedValue;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects;
+import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Objects;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Keeps watch over an Iceberg table and continuously outputs a range of 
snapshots, at the specified
+ * interval.
+ *
+ * <p>A downstream transform will create a list of read tasks for each range.
+ */
+class WatchForSnapshots extends PTransform<PBegin, PCollection<KV<String, 
List<SnapshotInfo>>>> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(WatchForSnapshots.class);
+  private final Duration pollInterval;
+  private final IcebergScanConfig scanConfig;
+
+  WatchForSnapshots(IcebergScanConfig scanConfig, Duration pollInterval) {
+    this.pollInterval = pollInterval;
+    this.scanConfig = scanConfig;
+  }
+
+  @Override
+  public PCollection<KV<String, List<SnapshotInfo>>> expand(PBegin input) {
+    return input
+        .apply(Create.of(scanConfig.getTableIdentifier()))
+        .apply(
+            "Scan Table Snapshots",
+            Watch.growthOf(new SnapshotPollFn(scanConfig))
+                .withPollInterval(pollInterval)
+                .withOutputCoder(ListCoder.of(SnapshotInfo.getCoder())))
+        .apply("Persist Snapshot Progress", ParDo.of(new 
PersistSnapshotProgress()));
+  }
+
+  /**
+   * Periodically scans the table for new snapshots, emitting a list for each 
new snapshot range.
+   *
+   * <p>This tracks progress locally but is not resilient to retries -- upon 
worker failure, it will
+   * restart from the initial starting strategy. Resilience is handled 
downstream by {@link
+   * PersistSnapshotProgress}.
+   */
+  private static class SnapshotPollFn extends Watch.Growth.PollFn<String, 
List<SnapshotInfo>> {
+    private final IcebergScanConfig scanConfig;
+    private @Nullable Long fromSnapshotId;
+
+    SnapshotPollFn(IcebergScanConfig scanConfig) {
+      this.scanConfig = scanConfig;
+    }
+
+    @Override
+    public PollResult<List<SnapshotInfo>> apply(String tableIdentifier, 
Context c) {
+      // fetch a fresh table to catch new snapshots
+      Table table =
+          TableCache.getRefreshed(tableIdentifier, 
scanConfig.getCatalogConfig().catalog());
+
+      @Nullable Long userSpecifiedToSnapshot = ReadUtils.getToSnapshot(table, 
scanConfig);
+      boolean isComplete = userSpecifiedToSnapshot != null;
+      if (fromSnapshotId == null) {
+        // first scan, initialize starting point with user config
+        fromSnapshotId = ReadUtils.getFromSnapshotExclusive(table, scanConfig);
+      }
+
+      Snapshot currentSnapshot = table.currentSnapshot();
+      if (currentSnapshot == null || 
Objects.equal(currentSnapshot.snapshotId(), fromSnapshotId)) {
+        // no new snapshots since last poll. return empty result.
+        return getPollResult(null, isComplete);
+      }
+
+      Long currentSnapshotId = currentSnapshot.snapshotId();
+      // if no upper bound is specified, we poll up to the current snapshot
+      long toSnapshotId = MoreObjects.firstNonNull(userSpecifiedToSnapshot, 
currentSnapshotId);
+
+      List<SnapshotInfo> snapshots =
+          ReadUtils.snapshotsBetween(table, tableIdentifier, fromSnapshotId, 
toSnapshotId);
+
+      fromSnapshotId = currentSnapshotId;
+      return getPollResult(snapshots, isComplete);
+    }
+
+    private PollResult<List<SnapshotInfo>> getPollResult(
+        @Nullable List<SnapshotInfo> snapshots, boolean isComplete) {
+      List<TimestampedValue<List<SnapshotInfo>>> timestampedSnapshots = new 
ArrayList<>(1);

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

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

Reply via email to