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


##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MetricConstants.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+public class MetricConstants {

Review Comment:
   nit:  `TableMaintenanceMetrics`?



##########
core/src/main/java/org/apache/iceberg/SerializableTable.java:
##########
@@ -397,6 +398,25 @@ private String errorMsg(String operation) {
     return String.format("Operation %s is not supported after the table is 
serialized", operation);
   }
 
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    SerializableTable other = (SerializableTable) o;
+    return Objects.equals(metadataFileLocation, other.metadataFileLocation);
+  }
+
+  @Override
+  public int hashCode() {
+    return metadataFileLocation.hashCode();

Review Comment:
   Can we avoid the requirement of `equals` and `hashCode`?
   
   Note that `metadataFileLocation` can be null. There were also discussion on 
potentially removing the metadata json file requirement from the spec.
   
   ```
     private String metadataFileLocation(Table table) {
       if (table instanceof HasTableOperations) {
         TableOperations ops = ((HasTableOperations) table).operations();
         return ops.current().metadataFileLocation();
       } else if (table instanceof BaseMetadataTable) {
         return ((BaseMetadataTable) 
table).table().operations().current().metadataFileLocation();
       } else {
         return null;
       }
     }
   ```



##########
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:
   why is this a `KeyedProcessFunction` after `keyBy`? I thought it will always 
be parallelism of 1. 



##########
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:
   do we need `List<TableChange>` here?  `TableChange` is mergeable. a single 
instance would be enough?



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java:
##########
@@ -0,0 +1,127 @@
+/*
+ * 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 lastTime, long currentTime) {

Review Comment:
   nit: may indicate where it is milli or micro in the arg names



##########
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:
   It is probably best not to use tag for this purpose. `Tag` is a named 
snapshot id. doesn't seem to be the right fit. 
   
   Alternative would be table property. I would love to hear others' inputs on 
the lock implementation.



##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java:
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.Objects;
+import org.apache.flink.annotation.Internal;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.relocated.com.google.common.base.MoreObjects;
+
+@Internal
+class Trigger {
+  private final long timestamp;
+  private final SerializableTable table;
+  private final Integer taskId;
+  private final boolean isCleanUp;
+
+  private Trigger(long timestamp, SerializableTable table, Integer taskId, 
boolean isCleanUp) {
+    this.timestamp = timestamp;
+    this.table = table;
+    this.taskId = taskId;

Review Comment:
   what is taskId for? it doesn't seem to be exposed to outside.



##########
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);

Review Comment:
   nit: `init` name is confusing. it is not initialization of this 
function/operator



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