Guosmilesmile commented on code in PR #15438:
URL: https://github.com/apache/iceberg/pull/15438#discussion_r2850976552


##########
flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManagerOperator.java:
##########
@@ -0,0 +1,323 @@
+/*
+ * 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.Serial;
+import java.util.List;
+import org.apache.flink.annotation.VisibleForTesting;
+import 
org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.metrics.Counter;
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.runtime.operators.coordination.OperatorEventGateway;
+import org.apache.flink.runtime.operators.coordination.OperatorEventHandler;
+import org.apache.flink.runtime.state.StateInitializationContext;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.StreamOperatorParameters;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
+import org.apache.iceberg.flink.maintenance.api.Trigger;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The TriggerManagerOperator itself holds the lock and registers a callback 
method with the
+ * coordinator. When a task finishes, it sends a signal from downstream to the 
coordinator to
+ * trigger this callback, allowing the TriggerManagerOperator to release the 
lock.
+ */
+class TriggerManagerOperator extends AbstractStreamOperator<Trigger>
+    implements OneInputStreamOperator<TableChange, Trigger>,
+        OperatorEventHandler,
+        ProcessingTimeCallback {
+
+  @Serial private static final long serialVersionUID = 1L;
+  private static final Logger LOG = 
LoggerFactory.getLogger(TriggerManagerOperator.class);
+
+  private final OperatorEventGateway operatorEventGateway;
+  private final List<String> maintenanceTaskNames;
+  private final List<TriggerEvaluator> evaluators;
+  private final long minFireDelayMs;
+  private final long lockCheckDelayMs;
+  private final String tableName;
+
+  private transient Counter rateLimiterTriggeredCounter;
+  private transient Counter concurrentRunThrottledCounter;
+  private transient Counter nothingToTriggerCounter;
+  private transient List<Counter> triggerCounters;
+  private transient ListState<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;
+  // To keep the task scheduling fair we keep the last triggered task position 
in memory.
+  // If we find a task to trigger, then we run it, but after it is finished, 
we start from the given
+  // position to prevent "starvation" of the tasks.
+  // When there is nothing to trigger, we start from the beginning, as the 
order of the tasks might
+  // be important (RewriteDataFiles first, and then RewriteManifestFiles later)
+  private transient int startsFrom = 0;
+  private transient boolean triggered = false;
+  private transient Long lockTime;
+  private transient boolean shouldRestoreTasks = false;
+
+  TriggerManagerOperator(
+      StreamOperatorParameters<Trigger> parameters,
+      OperatorEventGateway operatorEventGateway,
+      List<String> maintenanceTaskNames,
+      List<TriggerEvaluator> evaluators,
+      long minFireDelayMs,
+      long lockCheckDelayMs,
+      String tableName) {
+    super();
+    Preconditions.checkArgument(
+        maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(),
+        "Invalid maintenance task names: null or empty");
+    Preconditions.checkArgument(
+        evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null 
or empty");
+    Preconditions.checkArgument(
+        maintenanceTaskNames.size() == evaluators.size(),
+        "Provide a name and evaluator for all of the maintenance 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 
ms.");
+
+    this.processingTimeService = parameters.getProcessingTimeService();

Review Comment:
   Add init `processingTimeService` from 
`parameters.getProcessingTimeService()`.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to