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


##########
flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java:
##########
@@ -0,0 +1,371 @@
+/*
+ * 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.Locale;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ThreadFactory;
+import java.util.function.Consumer;
+import javax.annotation.Nonnull;
+import org.apache.flink.annotation.Experimental;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
+import org.apache.flink.runtime.operators.coordination.OperatorEvent;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FatalExitExceptionHandler;
+import org.apache.flink.util.FlinkRuntimeException;
+import 
org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Experimental
+@Internal
+public class TableMaintenanceCoordinator implements OperatorCoordinator {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TableMaintenanceCoordinator.class);
+
+  private final String operatorName;
+  private final Context context;
+
+  private final ExecutorService coordinatorExecutor;
+  private final CoordinatorExecutorThreadFactory coordinatorThreadFactory;
+  private transient boolean started;
+  private final transient SubtaskGateways subtaskGateways;
+  private static final Map<String, Consumer<LockReleasedEvent>> 
LOCK_RELEASE_CONSUMERS =
+      Maps.newConcurrentMap();
+  private transient List<LockReleasedEvent> pendingReleaseEvents = 
Lists.newArrayList();
+
+  public TableMaintenanceCoordinator(String operatorName, Context context) {
+    this.operatorName = operatorName;
+    this.context = context;
+
+    this.coordinatorThreadFactory =
+        new CoordinatorExecutorThreadFactory(
+            "TableMaintenanceCoordinator-" + operatorName, 
context.getUserCodeClassloader());
+    this.coordinatorExecutor = 
Executors.newSingleThreadExecutor(coordinatorThreadFactory);
+    this.subtaskGateways = new SubtaskGateways(operatorName, 
context.currentParallelism());
+    LOG.info("Created TableMaintenanceCoordinator: {}", operatorName);
+  }
+
+  @Override
+  public void start() throws Exception {
+    LOG.info("Starting TableMaintenanceCoordinator: {}", operatorName);
+    this.started = true;
+  }
+
+  @Override
+  public void close() throws Exception {
+    coordinatorExecutor.shutdown();
+    this.started = false;
+    LOG.info("Closed TableMaintenanceCoordinator: {}", operatorName);
+    LOCK_RELEASE_CONSUMERS.clear();
+    pendingReleaseEvents.clear();
+  }
+
+  @Override
+  public void handleEventFromOperator(int subtask, int attemptNumber, 
OperatorEvent event) {
+    runInCoordinatorThread(
+        () -> {
+          LOG.debug(
+              "Handling event from subtask {} (#{}) of {}: {}",
+              subtask,
+              attemptNumber,
+              operatorName,
+              event);
+          if (event instanceof LockRegisterEvent) {
+            registerTriggerManagerReceiveReleaseEvent((LockRegisterEvent) 
event);
+          } else if (event instanceof LockReleasedEvent) {
+            handleReleaseLock((LockReleasedEvent) event);
+          } else {
+            throw new IllegalArgumentException(
+                "Invalid operator event type: " + 
event.getClass().getCanonicalName());
+          }
+        },
+        String.format(
+            Locale.ROOT,
+            "handling operator event %s from subtask %d (#%d)",
+            event.getClass(),
+            subtask,
+            attemptNumber));
+  }
+
+  @SuppressWarnings("FutureReturnValueIgnored")
+  private void registerTriggerManagerReceiveReleaseEvent(LockRegisterEvent 
lockRegisterEvent) {
+    LOCK_RELEASE_CONSUMERS.put(
+        lockRegisterEvent.lockId(),
+        lock -> {
+          LOG.info(
+              "Send release event for lock id {}, timestamp: {} to Operator 
{}",
+              lock.lockId(),
+              lock.timestamp(),
+              operatorName);
+          this.subtaskGateways.getSubtaskGateway(0).sendEvent(lock);
+        });
+
+    if (!pendingReleaseEvents.isEmpty()) {
+      pendingReleaseEvents.forEach(this::handleReleaseLock);
+      pendingReleaseEvents.clear();
+    }
+  }
+
+  /** Release the lock and optionally trigger the next pending task. */
+  @VisibleForTesting
+  void handleReleaseLock(LockReleasedEvent lockReleasedEvent) {
+    if (LOCK_RELEASE_CONSUMERS.containsKey(lockReleasedEvent.lockId())) {
+      
LOCK_RELEASE_CONSUMERS.get(lockReleasedEvent.lockId()).accept(lockReleasedEvent);
+      LOG.info(
+          "Send release event for lock id {}, timestamp: {}",
+          lockReleasedEvent.lockId(),
+          lockReleasedEvent.timestamp());
+    } else {
+      pendingReleaseEvents.add(lockReleasedEvent);
+      LOG.info(
+          "No consumer for lock id {}, timestamp: {}",
+          lockReleasedEvent.lockId(),
+          lockReleasedEvent.timestamp());
+    }
+  }
+
+  @Override
+  public void checkpointCoordinator(long checkpointId, 
CompletableFuture<byte[]> resultFuture) {
+    // We don’t need to track how many locks are currently held, because when 
recovering from state,
+    // a `recover lock` will be issued to ensure all tasks finish running and 
then release all
+    // locks.
+    // The `TriggerManagerOperator` already keeps the `TableChange` state and 
related information,
+    // so there’s no need to store additional state here.
+    runInCoordinatorThread(
+        () -> {
+          resultFuture.complete(new byte[0]);
+        },
+        String.format(Locale.ROOT, "taking checkpoint %d", checkpointId));
+  }

Review Comment:
   Regarding recovery strategies for stateful tasks, these are two ideas:
   
   1. Current approach
   
   The `TriggerManagerOperator` does not persist the historical lock state. 
Instead, upon recovery it acquires a new lock and sends a recover trigger 
downstream. Once all tasks have finished, the downstream will release the lock.
   
   2. Alternative approach
   
   If we persist the lock state in `TriggerManagerOperator`, then upon task 
recovery there are two scenarios:
   
   1. Stateless tasks will not release the lock.
   2. Stateful tasks will release the lock when the task completes.
   
   In that case, we would need to determine explicitly, based on the task type, 
whether we should manually trigger a lock release during recovery. Stateful 
tasks must not trigger this manually. This feels like it would couple the 
design more tightly to the task implementation, and each task would need to be 
modified. By contrast, approach 1 looks more generic and reusable.
   
   Is my understanding correct? I’d like to hear your thoughts.@mxm @pvary 



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