pvary commented on code in PR #10308:
URL: https://github.com/apache/iceberg/pull/10308#discussion_r1605953735


##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java:
##########
@@ -0,0 +1,200 @@
+/*
+ * 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.iceberg.flink.maintenance.operator;
+
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.source.Boundedness;
+import org.apache.flink.api.connector.source.SourceReader;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import 
org.apache.flink.api.connector.source.util.ratelimit.RateLimitedSourceReader;
+import org.apache.flink.api.connector.source.util.ratelimit.RateLimiter;
+import 
org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy;
+import org.apache.flink.core.io.SimpleVersionedSerializer;
+import org.apache.flink.core.memory.DataInputDeserializer;
+import org.apache.flink.core.memory.DataOutputSerializer;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Monitors an Iceberg table for changes */
+public class MonitorSource extends SingleThreadedIteratorSource<TableChange> {
+  private static final Logger LOG = 
LoggerFactory.getLogger(MonitorSource.class);
+
+  private final TableLoader tableLoader;
+  private final RateLimiterStrategy rateLimiterStrategy;
+  private final long maxReadBack;
+
+  /**
+   * Creates a {@link org.apache.flink.api.connector.source.Source} which 
monitors an Iceberg table
+   * for changes.
+   *
+   * @param tableLoader used for accessing the table
+   * @param rateLimiterStrategy limits the frequency the table is checked
+   * @param maxReadBack sets the number of snapshots read before stopping 
change collection
+   */
+  public MonitorSource(

Review Comment:
   The `TriggerManager` doesn't do any scanning. It's only using the 
`TableChange` events to decide if the Maintenance Tasks are need to be started, 
or not. After the decision, we create an object with the current snapshot of 
the table, and with a current timestamp (this will be the actual Trigger), but 
we still don't do any table scanning (just read the current snapshot of the 
table). Only the actual Maintenance Tasks will read/scan the tables, but this 
only happens when they are triggered, and they are usually done in a way, which 
is very different than the `MonitorSource` scans.
   Also we would like to reuse the `TriggerManager` logic in the 
PostCommitTopology and when the maintenance is done in a separate job. We can 
separate out the `TriggerManager` to just a java code, but I feel that using 
the Flink provided operator abstraction is a nice way to do it, especially 
since we can make sure, that the operators are chained.



-- 
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: issues-unsubscr...@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to