pvary commented on code in PR #15151: URL: https://github.com/apache/iceberg/pull/15151#discussion_r2832082572
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/BaseCoordinator.java: ########## @@ -0,0 +1,357 @@ +/* + * 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.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +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.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; + +/** + * Base coordinator for table maintenance operators. Provides common functionality for thread + * management, subtask gateway management, and checkpoint handling. + */ +@Internal +public abstract class BaseCoordinator implements OperatorCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(BaseCoordinator.class); + + private final String operatorName; + private final Context context; + + private final ExecutorService coordinatorExecutor; + private boolean started; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private final SubtaskGateways subtaskGateways; + private static final Map<String, Consumer<LockReleaseEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + private static final List<LockReleaseEvent> PENDING_RELEASE_EVENTS = Lists.newArrayList(); + + protected BaseCoordinator(String operatorName, Context context) { + this.operatorName = operatorName; + this.context = context; + + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "Coordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + LOG.info("Created coordinator: {}", operatorName); + } + + void registerLock(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()); + subtaskGateways().getSubtaskGateway(0).sendEvent(lock); + }); + + synchronized (PENDING_RELEASE_EVENTS) { + if (!PENDING_RELEASE_EVENTS.isEmpty()) { + PENDING_RELEASE_EVENTS.forEach(this::handleReleaseLock); + PENDING_RELEASE_EVENTS.clear(); + } + } + } + + @VisibleForTesting + void handleReleaseLock(LockReleaseEvent lockReleaseEvent) { + synchronized (PENDING_RELEASE_EVENTS) { + if (LOCK_RELEASE_CONSUMERS.containsKey(lockReleaseEvent.lockId())) { + LOCK_RELEASE_CONSUMERS.get(lockReleaseEvent.lockId()).accept(lockReleaseEvent); + LOG.info( + "Send release event for lock id {}, timestamp: {}", + lockReleaseEvent.lockId(), + lockReleaseEvent.timestamp()); + } else { + PENDING_RELEASE_EVENTS.add(lockReleaseEvent); + LOG.info( + "No consumer for lock id {}, timestamp: {}", + lockReleaseEvent.lockId(), + lockReleaseEvent.timestamp()); + } + } + } + + @Override + public void start() throws Exception { + LOG.info("Starting coordinator: {}", operatorName); + this.started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + this.started = false; + synchronized (PENDING_RELEASE_EVENTS) { + LOCK_RELEASE_CONSUMERS.clear(); + PENDING_RELEASE_EVENTS.clear(); + } + + LOG.info("Closed coordinator: {}", operatorName); + } + + @Override + public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> resultFuture) + throws Exception { + runInCoordinatorThread( + () -> resultFuture.complete(new byte[0]), + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); + } + + @Override + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { + Preconditions.checkState( + !started, "The coordinator %s can only be reset if it was not yet started", operatorName); + LOG.info("Reset to checkpoint {}", checkpointId); + synchronized (PENDING_RELEASE_EVENTS) { + LOCK_RELEASE_CONSUMERS.clear(); + PENDING_RELEASE_EVENTS.clear(); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) {} + + @Override + public void subtaskReset(int subtask, long checkpointId) { + runInCoordinatorThread( + () -> { + LOG.info("Subtask {} is reset to checkpoint {}", subtask, checkpointId); + Preconditions.checkState(coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.reset(subtask); + }, + String.format( + Locale.ROOT, "handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + } + + @Override + public void executionAttemptFailed(int subtask, int attemptNumber, Throwable reason) { + runInCoordinatorThread( + () -> { + LOG.info( + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", + subtask, + attemptNumber, + operatorName); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); + }, + String.format(Locale.ROOT, "handling subtask %d (#%d) failure", subtask, attemptNumber)); + } + + @Override + public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway gateway) { + Preconditions.checkArgument(subtask == gateway.getSubtask()); + Preconditions.checkArgument(attemptNumber == gateway.getExecution().getAttemptNumber()); + runInCoordinatorThread( + () -> { + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.registerSubtaskGateway(gateway); + }, + String.format( + Locale.ROOT, + "making event gateway to subtask %d (#%d) available", + subtask, + attemptNumber)); + } + + protected CoordinatorExecutorThreadFactory coordinatorThreadFactory() { + return coordinatorThreadFactory; + } + + protected SubtaskGateways subtaskGateways() { + return subtaskGateways; + } + + protected String operatorName() { + return operatorName; + } + + protected void runInCoordinatorThread(Runnable runnable, String actionString) { + ensureStarted(); + coordinatorExecutor.execute( + () -> { + try { + runnable.run(); + } catch (Throwable t) { + LOG.error( + "Uncaught exception in coordinator while {}: {}", actionString, t.getMessage(), t); + context.failJob(t); + } + }); + } + + @VisibleForTesting + void callInCoordinatorThread(Callable<Void> callable, String errorMessage) { + ensureStarted(); + // Ensure the task is done by the coordinator executor. + if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) { + try { + Callable<Void> guardedCallable = + () -> { + try { + return callable.call(); + } catch (Throwable t) { + LOG.error("Uncaught Exception in coordinator {} executor", operatorName, t); + ExceptionUtils.rethrowException(t); + return null; + } + }; + + coordinatorExecutor.submit(guardedCallable).get(); + } catch (InterruptedException | ExecutionException e) { + throw new FlinkRuntimeException(errorMessage, e); + } + } else { + try { + callable.call(); + } catch (Throwable t) { + LOG.error("Uncaught Exception in coordinator {} executor", operatorName, t); + throw new FlinkRuntimeException(errorMessage, t); + } + } + } + + private void ensureStarted() { + Preconditions.checkState(started, "The coordinator has not started yet."); + } + + /** Inner class to manage subtask gateways. */ + protected static class SubtaskGateways { Review Comment: This could be a private `record` instead of a class -- 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]
