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


##########
flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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.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;
+
+/**
+ * TriggerManager starts the Maintenance Tasks by emitting {@link Trigger} 
messages which are
+ * calculated based on the incoming {@link TableChange} messages. The 
TriggerManager keeps track of
+ * the changes since the last run of the Maintenance Tasks and triggers a new 
run based on the
+ * result of the {@link TriggerEvaluator}.
+ *
+ * <p>The TriggerManager prevents overlapping Maintenance Task runs using 
{@link
+ * org.apache.iceberg.flink.maintenance.operator.TriggerLockFactory.Lock}.
+ *
+ * <p>The TriggerManager should run as a global operator. {@link 
KeyedProcessFunction} is used, so
+ * the timer functions are available, but the key is not used.
+ */
+@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> nextEvaluationTimeState;
+  private transient ListState<TableChange> accumulatedChangesState;
+  private transient ListState<Long> lastTriggerTimesState;
+  private transient Long nextEvaluationTime;
+  private transient List<TableChange> accumulatedChanges;
+  private transient List<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 
1ms.");
+
+    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.nextEvaluationTimeState =
+        getRuntimeContext()
+            .getState(new 
ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG));
+    this.accumulatedChangesState =
+        getRuntimeContext()
+            .getListState(
+                new ListStateDescriptor<>(
+                    "triggerManagerAccumulatedChange", 
TypeInformation.of(TableChange.class)));
+    this.lastTriggerTimesState =
+        getRuntimeContext()
+            .getListState(new 
ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG));
+
+    tableLoader.open();
+  }
+
+  @Override
+  public void snapshotState(FunctionSnapshotContext context) throws Exception {
+    if (inited) {
+      // Only store state if initialized
+      nextEvaluationTimeState.update(nextEvaluationTime);
+      accumulatedChangesState.update(accumulatedChanges);
+      lastTriggerTimesState.update(lastTriggerTimes);
+    }
+  }
+
+  @Override
+  public void initializeState(FunctionInitializationContext context) throws 
Exception {
+    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());
+
+    accumulatedChanges.forEach(tableChange -> tableChange.merge(change));
+
+    long current = ctx.timerService().currentProcessingTime();
+    Long nextTime = nextEvaluationTime;
+    if (nextTime == null) {
+      checkAndFire(current, ctx.timerService(), out);
+    } else {
+      LOG.info(
+          "Trigger manager rate limiter triggered current: {}, next: {}, 
accumulated changes: {}",
+          current,
+          nextTime,
+          accumulatedChanges);
+      rateLimiterTriggeredCounter.inc();
+    }
+  }
+
+  @Override
+  public void onTimer(long timestamp, OnTimerContext ctx, Collector<Trigger> 
out) throws Exception {
+    init(out, ctx.timerService());
+    this.nextEvaluationTime = null;
+    checkAndFire(ctx.timerService().currentProcessingTime(), 
ctx.timerService(), out);
+  }
+
+  private void checkAndFire(long current, TimerService timerService, 
Collector<Trigger> out) {
+    if (isCleanUp) {
+      if (recoveryLock.isHeld()) {
+        LOG.debug("The cleanup lock is still held at {}", current);
+        schedule(timerService, current + lockCheckDelayMs);

Review Comment:
   We come here only by 2 routes:
   - `processElement` - in this case `nextTime`/`nextEvaluationTime` is null - 
no previous timer scheduled
   - `onTimer` - in this case the previous timer has fired, so no previous 
timer scheduled



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