stevenzwu commented on code in PR #10484:
URL: https://github.com/apache/iceberg/pull/10484#discussion_r1648221004


##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JVMBasedLockFactory.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.util.concurrent.Semaphore;
+import org.apache.flink.annotation.Internal;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The locks are based on static {@link Semaphore} objects. We expect that the 
{@link
+ * TriggerManager} and the LockRemover operators will be placed on the same 
TaskManager (JVM), as
+ * they are both global. In this case JVM based locking should be enough to 
allow communication
+ * between the operators.
+ */
+@Internal
+public class JVMBasedLockFactory implements TriggerLockFactory {

Review Comment:
   nit: JVM seems redundant as this is Java code. maybe `SemaphoreLockFactory` 
or `InMemorySemaphoreLockFactory`?



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(TableMaintenanceMetrics.GROUP_KEY, name)
+                        .counter(TableMaintenanceMetrics.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}", context.isRestored());
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;

Review Comment:
   nit: `shouldCleanUp` seems more accurate. `isCleanUp` makes more sense in 
the `Trigger` class



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(TableMaintenanceMetrics.GROUP_KEY, name)
+                        .counter(TableMaintenanceMetrics.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}", context.isRestored());
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    init(out, ctx.timerService());
+
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime.value();
+
+    // Add the new changes to the already accumulated ones
+    List<TableChange> accumulated = 
Lists.newArrayList(accumulatedChanges.get());
+    accumulated.forEach(tableChange -> tableChange.merge(change));
+    accumulatedChanges.update(accumulated);
+
+    if (nextTime == null) {
+      checkAndFire(current, ctx.timerService(), out);
+    } else {
+      LOG.info(

Review Comment:
   do we need to check if `nextTime` is larger than the `current` time here?



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(TableMaintenanceMetrics.GROUP_KEY, name)
+                        .counter(TableMaintenanceMetrics.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}", context.isRestored());
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    init(out, ctx.timerService());
+
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime.value();
+
+    // Add the new changes to the already accumulated ones
+    List<TableChange> accumulated = 
Lists.newArrayList(accumulatedChanges.get());
+    accumulated.forEach(tableChange -> tableChange.merge(change));
+    accumulatedChanges.update(accumulated);
+
+    if (nextTime == null) {
+      checkAndFire(current, ctx.timerService(), out);
+    } else {
+      LOG.info(
+          "Trigger manager rate limiter triggered current: {}, next: {}, 
accumulated changes: {}",
+          current,
+          nextTime,
+          accumulated);
+      rateLimiterTriggeredCounter.inc();
+    }
+  }
+
+  @Override
+  public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> 
out) throws Exception {
+    init(out, ctx.timerService());
+    nextEvaluationTime.clear();
+    checkAndFire(ctx.timerService().currentProcessingTime(), 
ctx.timerService(), out);
+  }
+
+  private void checkAndFire(long current, TimerService timerService, 
Collector<Trigger> out)
+      throws Exception {
+    if (isCleanUp) {
+      if (recoveryLock.isHeld()) {
+        LOG.debug("The cleanup lock is still held at {}", current);
+        schedule(timerService, current + lockCheckDelayMs);
+        return;
+      } else {
+        LOG.info("The cleanup is finished at {}", current);
+        isCleanUp = false;
+      }
+    }
+
+    List<TableChange> changes = Lists.newArrayList(accumulatedChanges.get());

Review Comment:
   nit: It is probably more common to keep these lists as class members. and in 
the `snapshotState` method, just update the state variable with the list.
   
   for this use case (low-frequency state read/write), it doesn't matter 
performance wise (either way).



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TagBasedLockFactory.java:
##########
@@ -0,0 +1,139 @@
+/*
+ * 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.util.Map;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.ManageSnapshots;
+import org.apache.iceberg.SnapshotRef;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.util.Tasks;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Iceberg table {@link ManageSnapshots#createTag(String, long)}/{@link
+ * ManageSnapshots#removeTag(String)} based lock implementation for {@link 
TriggerLockFactory}.
+ */
+@Internal
+public class TagBasedLockFactory implements TriggerLockFactory {

Review Comment:
   should we avoid `TagBasedLockFactory` to start with? I am still a bit 
concerned about using Tag as distributed lock mechanism just semantically.
   
   `SemaphoreLock` is a simple and good implementation to provide. It does has 
some limitations. E.g., we probably can't specify the slot sharing group to 
control the colocation for Flink SQL jobs?
   
   Another potential implementation is using k8s ConfigMap/Etcd. but we 
probably don't want to pull in the k8s Java SDK dep in Flink module.



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private final boolean clearLocks;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs,
+      boolean clearLocks) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");

Review Comment:
   I am not sure if we are talking about the same thing. I was saying moving 
the `String taskName` into the `TriggerEvaluator` class. this way, we only need 
`List<TriggerEvaluator> evaluators`. Then we don't need the check on 
`taskNames.size() == evaluators.size()`



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");

Review Comment:
   nit: add the time unit `1 ms`. same for the error msg below



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.Serializable;
+import java.time.Duration;
+import java.util.List;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Internal
+class TriggerEvaluator implements Serializable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerEvaluator.class);
+  private final List<Predicate> predicates;
+
+  private TriggerEvaluator(List<Predicate> predicates) {
+    Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 
condition.");
+
+    this.predicates = predicates;
+  }
+
+  boolean check(TableChange event, long lastTimeMs, long currentTimeMs) {
+    boolean result =
+        predicates.stream()
+            .anyMatch(
+                p -> {
+                  try {
+                    return p.evaluate(event, lastTimeMs, currentTimeMs);
+                  } catch (Exception e) {
+                    throw new RuntimeException("Error accessing state", e);
+                  }
+                });
+    LOG.debug(
+        "Checking event: {}, at {}, last: {} with result: {}",
+        event,
+        currentTimeMs,
+        lastTimeMs,
+        result);
+    return result;
+  }
+
+  static class Builder implements Serializable {
+    private Integer commitNumber;
+    private Integer fileNumber;
+    private Long fileSize;
+    private Integer deleteFileNumber;

Review Comment:
   do we need `deleteFileBytes`?
   
   I am also wondering if we need to separate out position delete and equality 
deletes for trigger evaluator.



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(TableMaintenanceMetrics.GROUP_KEY, name)
+                        .counter(TableMaintenanceMetrics.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}", context.isRestored());
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    init(out, ctx.timerService());
+
+    long current = ctx.timerService().currentProcessingTime();

Review Comment:
   nit: move the variable definitions right before the usage for readability



##########
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java:
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+class ConstantsForTests {
+  public static final long EVENT_TIME = 10L;

Review Comment:
   I am not sure these constants improve readability. e.g., what is the 
relationship (comparison) btw EVENT_TIME and EVENT_TIME_2? it would be easier 
to read the literals directly.
   
   similar comment for dummy name.



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>

Review Comment:
   thanks for verifying. please add the context to the class javadoc, which is 
empty currently.



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private final boolean clearLocks;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs,
+      boolean clearLocks) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+    this.clearLocks = clearLocks;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(MetricConstants.GROUP_KEY, name)
+                        .counter(MetricConstants.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}, clearLocks: {}", 
context.isRestored(), clearLocks);
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    } else if (clearLocks) {
+      // Remove old lock if we are not restoring the job
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime.value();
+    init(out, ctx.timerService(), current);
+
+    // Add the new changes to the already accumulated ones
+    List<TableChange> accumulated = 
Lists.newArrayList(accumulatedChanges.get());

Review Comment:
   oh. I got it now. the list is has one `TableChange` accumulation for each 
maintenance task



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java:
##########
@@ -0,0 +1,128 @@
+/*
+ * 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.Serializable;
+import java.time.Duration;
+import java.util.List;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Internal
+class TriggerEvaluator implements Serializable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerEvaluator.class);
+  private final List<Predicate> predicates;
+
+  private TriggerEvaluator(List<Predicate> predicates) {
+    Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 
condition.");
+
+    this.predicates = predicates;
+  }
+
+  boolean check(TableChange event, long lastTimeMs, long currentTimeMs) {
+    boolean result =
+        predicates.stream()
+            .anyMatch(
+                p -> {
+                  try {
+                    return p.evaluate(event, lastTimeMs, currentTimeMs);
+                  } catch (Exception e) {
+                    throw new RuntimeException("Error accessing state", e);
+                  }
+                });
+    LOG.debug(
+        "Checking event: {}, at {}, last: {} with result: {}",
+        event,
+        currentTimeMs,
+        lastTimeMs,
+        result);
+    return result;
+  }
+
+  static class Builder implements Serializable {
+    private Integer commitNumber;
+    private Integer fileNumber;

Review Comment:
   nit: can we use variable naming pattern similar to existing code? Here are 
some from `PartitionsTable` class. 
   
   ```
       private int dataFileCount;
       private long dataFileSizeInBytes;
       private long posDeleteRecordCount;
       private int posDeleteFileCount;
       private long eqDeleteRecordCount;
       private int eqDeleteFileCount;
   ```



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(TableMaintenanceMetrics.GROUP_KEY, name)
+                        .counter(TableMaintenanceMetrics.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}", context.isRestored());
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    init(out, ctx.timerService());
+
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime.value();
+
+    // Add the new changes to the already accumulated ones
+    List<TableChange> accumulated = 
Lists.newArrayList(accumulatedChanges.get());
+    accumulated.forEach(tableChange -> tableChange.merge(change));
+    accumulatedChanges.update(accumulated);
+
+    if (nextTime == null) {
+      checkAndFire(current, ctx.timerService(), out);
+    } else {
+      LOG.info(
+          "Trigger manager rate limiter triggered current: {}, next: {}, 
accumulated changes: {}",
+          current,
+          nextTime,
+          accumulated);
+      rateLimiterTriggeredCounter.inc();
+    }
+  }
+
+  @Override
+  public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> 
out) throws Exception {
+    init(out, ctx.timerService());
+    nextEvaluationTime.clear();
+    checkAndFire(ctx.timerService().currentProcessingTime(), 
ctx.timerService(), out);
+  }
+
+  private void checkAndFire(long current, TimerService timerService, 
Collector<Trigger> out)
+      throws Exception {
+    if (isCleanUp) {
+      if (recoveryLock.isHeld()) {
+        LOG.debug("The cleanup lock is still held at {}", current);
+        schedule(timerService, current + lockCheckDelayMs);
+        return;
+      } else {
+        LOG.info("The cleanup is finished at {}", current);
+        isCleanUp = false;
+      }
+    }
+
+    List<TableChange> changes = Lists.newArrayList(accumulatedChanges.get());
+    List<Long> times = Lists.newArrayList(lastTriggerTimes.get());
+    Integer taskToStart = nextTrigger(evaluators, changes, times, current, 
startsFrom);
+    if (taskToStart == null) {
+      // Nothing to execute
+      if (startsFrom == 0) {
+        nothingToTriggerCounter.inc();
+        LOG.debug("Nothing to execute at {} for collected: {}", current, 
changes);
+      }
+
+      startsFrom = 0;

Review Comment:
   in case of nothing to trigger, why do we need to reset the cursor/position 
to 0?



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JVMBasedLockFactory.java:
##########
@@ -0,0 +1,83 @@
+/*
+ * 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.util.concurrent.Semaphore;
+import org.apache.flink.annotation.Internal;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The locks are based on static {@link Semaphore} objects. We expect that the 
{@link
+ * TriggerManager} and the LockRemover operators will be placed on the same 
TaskManager (JVM), as
+ * they are both global. In this case JVM based locking should be enough to 
allow communication

Review Comment:
   just global operator doesn't guarantee the colocation. we probably should 
call out the Flink slot sharing group. some brief example would be helpful.



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(TableMaintenanceMetrics.GROUP_KEY, name)
+                        .counter(TableMaintenanceMetrics.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}", context.isRestored());
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    init(out, ctx.timerService());
+
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime.value();
+
+    // Add the new changes to the already accumulated ones
+    List<TableChange> accumulated = 
Lists.newArrayList(accumulatedChanges.get());
+    accumulated.forEach(tableChange -> tableChange.merge(change));
+    accumulatedChanges.update(accumulated);
+
+    if (nextTime == null) {
+      checkAndFire(current, ctx.timerService(), out);
+    } else {
+      LOG.info(
+          "Trigger manager rate limiter triggered current: {}, next: {}, 
accumulated changes: {}",
+          current,
+          nextTime,
+          accumulated);
+      rateLimiterTriggeredCounter.inc();
+    }
+  }
+
+  @Override
+  public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> 
out) throws Exception {
+    init(out, ctx.timerService());
+    nextEvaluationTime.clear();
+    checkAndFire(ctx.timerService().currentProcessingTime(), 
ctx.timerService(), out);
+  }
+
+  private void checkAndFire(long current, TimerService timerService, 
Collector<Trigger> out)
+      throws Exception {
+    if (isCleanUp) {
+      if (recoveryLock.isHeld()) {
+        LOG.debug("The cleanup lock is still held at {}", current);
+        schedule(timerService, current + lockCheckDelayMs);
+        return;
+      } else {
+        LOG.info("The cleanup is finished at {}", current);
+        isCleanUp = false;
+      }
+    }
+
+    List<TableChange> changes = Lists.newArrayList(accumulatedChanges.get());
+    List<Long> times = Lists.newArrayList(lastTriggerTimes.get());
+    Integer taskToStart = nextTrigger(evaluators, changes, times, current, 
startsFrom);
+    if (taskToStart == null) {
+      // Nothing to execute
+      if (startsFrom == 0) {
+        nothingToTriggerCounter.inc();
+        LOG.debug("Nothing to execute at {} for collected: {}", current, 
changes);
+      }
+
+      startsFrom = 0;
+      return;
+    }
+
+    if (lock.tryLock()) {
+      TableChange change = changes.get(taskToStart);
+      SerializableTable table =

Review Comment:
   for each `Trigger` record, we are loading the table from catalog. why do we 
let downstream operator load the table directly? i.e. downstream operator gets 
the `TableLoader` and we avoid including `SerializableTable` to the `Trigger` 
object?



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(
+                TableMaintenanceMetrics.GROUP_KEY, 
TableMaintenanceMetrics.GROUP_VALUE_DEFAULT)
+            .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(TableMaintenanceMetrics.GROUP_KEY, name)
+                        .counter(TableMaintenanceMetrics.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}", context.isRestored());
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    init(out, ctx.timerService());
+
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime.value();
+
+    // Add the new changes to the already accumulated ones
+    List<TableChange> accumulated = 
Lists.newArrayList(accumulatedChanges.get());
+    accumulated.forEach(tableChange -> tableChange.merge(change));
+    accumulatedChanges.update(accumulated);
+
+    if (nextTime == null) {
+      checkAndFire(current, ctx.timerService(), out);
+    } else {
+      LOG.info(
+          "Trigger manager rate limiter triggered current: {}, next: {}, 
accumulated changes: {}",
+          current,
+          nextTime,
+          accumulated);
+      rateLimiterTriggeredCounter.inc();
+    }
+  }
+
+  @Override
+  public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> 
out) throws Exception {
+    init(out, ctx.timerService());
+    nextEvaluationTime.clear();
+    checkAndFire(ctx.timerService().currentProcessingTime(), 
ctx.timerService(), out);
+  }
+
+  private void checkAndFire(long current, TimerService timerService, 
Collector<Trigger> out)
+      throws Exception {
+    if (isCleanUp) {
+      if (recoveryLock.isHeld()) {
+        LOG.debug("The cleanup lock is still held at {}", current);
+        schedule(timerService, current + lockCheckDelayMs);
+        return;
+      } else {
+        LOG.info("The cleanup is finished at {}", current);
+        isCleanUp = false;
+      }
+    }
+
+    List<TableChange> changes = Lists.newArrayList(accumulatedChanges.get());
+    List<Long> times = Lists.newArrayList(lastTriggerTimes.get());
+    Integer taskToStart = nextTrigger(evaluators, changes, times, current, 
startsFrom);
+    if (taskToStart == null) {
+      // Nothing to execute
+      if (startsFrom == 0) {
+        nothingToTriggerCounter.inc();
+        LOG.debug("Nothing to execute at {} for collected: {}", current, 
changes);
+      }
+
+      startsFrom = 0;
+      return;
+    }
+
+    if (lock.tryLock()) {
+      TableChange change = changes.get(taskToStart);
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());
+      out.collect(Trigger.create(current, table, taskToStart));
+      LOG.debug("Fired event with time: {}, collected: {} for {}", current, 
change, table.name());
+      triggerCounters.get(taskToStart).inc();
+      changes.set(taskToStart, TableChange.empty());
+      accumulatedChanges.update(changes);
+      schedule(timerService, current + minFireDelayMs);
+      startsFrom = taskToStart + 1;

Review Comment:
   might be a bit more readable to do the `mod` here (than in the `nextTrigger` 
method)



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private final boolean clearLocks;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs,
+      boolean clearLocks) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+    this.clearLocks = clearLocks;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(MetricConstants.GROUP_KEY, name)
+                        .counter(MetricConstants.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}, clearLocks: {}", 
context.isRestored(), clearLocks);
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;
+    } else if (clearLocks) {
+      // Remove old lock if we are not restoring the job
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void processElement(TableChange change, Context ctx, 
Collector<Trigger> out)
+      throws Exception {
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime.value();
+    init(out, ctx.timerService(), current);
+
+    // Add the new changes to the already accumulated ones
+    List<TableChange> accumulated = 
Lists.newArrayList(accumulatedChanges.get());
+    accumulated.forEach(tableChange -> tableChange.merge(change));
+    accumulatedChanges.update(accumulated);
+
+    if (nextTime == null) {
+      checkAndFire(current, ctx.timerService(), out);
+    } else {
+      LOG.info(
+          "Trigger manager rate limiter triggered current: {}, next: {}, 
accumulated changes: {}",
+          current,
+          nextTime,
+          accumulated);
+      rateLimiterTriggeredCounter.inc();
+    }
+  }
+
+  @Override
+  public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> 
out) throws Exception {
+    init(out, ctx.timerService(), timestamp);
+    nextEvaluationTime.clear();
+    checkAndFire(ctx.timerService().currentProcessingTime(), 
ctx.timerService(), out);
+  }
+
+  private void checkAndFire(long current, TimerService timerService, 
Collector<Trigger> out)
+      throws Exception {
+    if (isCleanUp) {
+      if (recoveryLock.isHeld()) {
+        LOG.debug("The cleanup lock is still held at {}", current);
+        schedule(timerService, current + lockCheckDelayMs);
+        return;
+      } else {
+        LOG.info("The cleanup is finished at {}", current);
+        isCleanUp = false;
+      }
+    }
+
+    List<TableChange> changes = Lists.newArrayList(accumulatedChanges.get());
+    List<Long> times = Lists.newArrayList(lastTriggerTimes.get());
+    Integer taskToStart = nextTrigger(evaluators, changes, times, current, 
startsFrom);
+    if (taskToStart == null) {
+      // Nothing to execute
+      if (startsFrom == 0) {
+        nothingToTriggerCounter.inc();
+        LOG.debug("Nothing to execute at {} for collected: {}", current, 
changes);
+      }
+
+      startsFrom = 0;
+      return;
+    }
+
+    if (lock.tryLock()) {
+      TableChange change = changes.get(taskToStart);
+      SerializableTable table =
+          (SerializableTable) 
SerializableTable.copyOf(tableLoader.loadTable());
+      out.collect(Trigger.create(current, table, taskToStart));
+      LOG.debug("Fired event with time: {}, collected: {} for {}", current, 
change, table.name());
+      triggerCounters.get(taskToStart).inc();
+      changes.set(taskToStart, TableChange.empty());
+      accumulatedChanges.update(changes);
+      schedule(timerService, current + minFireDelayMs);
+      startsFrom = taskToStart + 1;
+    } else {
+      // The lock is already held by someone
+      LOG.info("Delaying task on failed lock check: {}", current);
+
+      startsFrom = taskToStart;
+      concurrentRunTriggeredCounter.inc();
+      schedule(timerService, current + lockCheckDelayMs);
+    }
+
+    timerService.registerProcessingTimeTimer(nextEvaluationTime.value());
+  }
+
+  private void schedule(TimerService timerService, long time) throws 
IOException {
+    nextEvaluationTime.update(time);
+    timerService.registerProcessingTimeTimer(time);
+  }
+
+  private static Integer nextTrigger(
+      List<TriggerEvaluator> evaluators,
+      List<TableChange> changes,
+      List<Long> lastTriggerTimes,
+      long currentTime,
+      int startPos) {
+    int normalizedStartingPos = startPos % evaluators.size();
+    int current = normalizedStartingPos;
+    do {
+      if (evaluators
+          .get(current)
+          .check(changes.get(current), lastTriggerTimes.get(current), 
currentTime)) {
+        return current;
+      }
+
+      current = (current + 1) % evaluators.size();
+    } while (current != normalizedStartingPos);
+
+    return null;
+  }
+
+  private void init(Collector<Trigger> out, TimerService timerService, long 
current)
+      throws Exception {
+    if (!inited) {
+      // Initialize with empty changes and current timestamp
+      if (!accumulatedChanges.get().iterator().hasNext()) {
+        List<TableChange> changes = 
Lists.newArrayListWithCapacity(evaluators.size());
+        List<Long> triggerTimes = 
Lists.newArrayListWithCapacity(evaluators.size());
+        for (int i = 0; i < evaluators.size(); ++i) {
+          changes.add(TableChange.empty());
+          triggerTimes.add(current);
+        }
+
+        accumulatedChanges.update(changes);
+        lastTriggerTimes.update(triggerTimes);
+      }
+
+      if (isCleanUp) {
+        // When the job state is restored, there could be ongoing tasks.
+        // To prevent collision with the new triggers the following is done:
+        //  - add a cleanup lock
+        //  - fire a clean-up trigger
+        // This ensures that the tasks of the previous trigger are executed, 
and the lock is removed
+        // in the end.
+        recoveryLock.tryLock();

Review Comment:
   we don't check the boolean return value from `tryLock`. is that correct?



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,295 @@
+/*
+ * 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.List;
+import java.util.stream.Collectors;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.TimerService;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
+import org.apache.flink.util.Collector;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** . */
+@Internal
+class TriggerManager extends KeyedProcessFunction<Boolean, TableChange, 
Trigger>
+    implements CheckpointedFunction {
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManager.class);
+
+  private final TableLoader tableLoader;
+  private final TriggerLockFactory lockFactory;
+  private final List<String> taskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private final boolean clearLocks;
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunTriggeredCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ValueState<Long> nextEvaluationTime;
+  private transient ListState<TableChange> accumulatedChanges;
+  private transient ListState<Long> lastTriggerTimes;
+  private transient TriggerLockFactory.Lock lock;
+  private transient TriggerLockFactory.Lock recoveryLock;
+  private transient boolean isCleanUp = false;
+  private transient boolean inited = false;
+  private transient int startsFrom = 0;
+
+  TriggerManager(
+      TableLoader tableLoader,
+      TriggerLockFactory lockFactory,
+      List<String> taskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs,
+      boolean clearLocks) {
+    Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
+    Preconditions.checkNotNull(lockFactory, "Lock factory should no be null");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Evaluators should not be 
empty");
+    Preconditions.checkArgument(
+        taskNames.size() == evaluators.size(), "Provide a name and evaluator 
for all of the tasks");
+    Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should 
be at least 1.");
+    Preconditions.checkArgument(
+        lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1.");
+
+    this.tableLoader = tableLoader;
+    this.lockFactory = lockFactory;
+    this.taskNames = taskNames;
+    this.evaluators = evaluators;
+    this.minFireDelayMs = minFireDelayMs;
+    this.lockCheckDelayMs = lockCheckDelayMs;
+    this.clearLocks = clearLocks;
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    this.rateLimiterTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.RATE_LIMITER_TRIGGERED);
+    this.concurrentRunTriggeredCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.CONCURRENT_RUN_TRIGGERED);
+    this.nothingToTriggerCounter =
+        getRuntimeContext()
+            .getMetricGroup()
+            .addGroup(MetricConstants.GROUP_KEY, 
MetricConstants.GROUP_VALUE_DEFAULT)
+            .counter(MetricConstants.NOTHING_TO_TRIGGER);
+    this.triggerCounters =
+        taskNames.stream()
+            .map(
+                name ->
+                    getRuntimeContext()
+                        .getMetricGroup()
+                        .addGroup(MetricConstants.GROUP_KEY, name)
+                        .counter(MetricConstants.TRIGGERED))
+            .collect(Collectors.toList());
+
+    this.nextEvaluationTime =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChanges =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimes =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    // Do nothing
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) {
+    LOG.info("Initializing state restored: {}, clearLocks: {}", 
context.isRestored(), clearLocks);
+    this.lock = lockFactory.createLock();
+    this.recoveryLock = lockFactory.createRecoveryLock();
+    if (context.isRestored()) {
+      isCleanUp = true;

Review Comment:
   nit: maybe add the context as code comment?



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