mxm commented on code in PR #15042: URL: https://github.com/apache/iceberg/pull/15042#discussion_r2690270593
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceCoordinator.java: ########## @@ -0,0 +1,379 @@ +/* + * 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.Locale; +import java.util.Map; +import java.util.Set; +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 javax.annotation.Nonnull; +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.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@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 Set<String> LOCK_HELD_SET = Sets.newConcurrentHashSet(); + private static final Set<String> RECOVER_LOCK_HELD_SET = Sets.newConcurrentHashSet(); + + 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_HELD_SET.clear(); + RECOVER_LOCK_HELD_SET.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 LockReleasedEvent) { + handlerReleaseLock((LockReleasedEvent) event); + } else if (event instanceof LockAcquiredEvent) { + handleLockAcquired((LockAcquiredEvent) event, subtask); + } 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 handleLockAcquired(LockAcquiredEvent event, int subtask) { + runInCoordinatorThread( + () -> { + String lockId = event.lockId(); + boolean isHeld = false; + if (event.isRecoverLock()) { + if (!RECOVER_LOCK_HELD_SET.contains(lockId)) { + RECOVER_LOCK_HELD_SET.add(lockId); + isHeld = true; + } + } else { + if (!LOCK_HELD_SET.contains(lockId)) { + LOCK_HELD_SET.add(lockId); + isHeld = true; + } + } + + LockAcquireResultEvent lockAcquireResultEvent = + new LockAcquireResultEvent(event.isRecoverLock(), isHeld, lockId); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(lockAcquireResultEvent); + }, + String.format( + Locale.ROOT, + "Failed to send operator %s coordinator to requesting subtask %d ", + operatorName, + subtask)); + } + + /** Release the lock and optionally trigger the next pending task. */ + private void handlerReleaseLock(LockReleasedEvent lockReleasedEvent) { Review Comment: ```suggestion private void handleReleaseLock(LockReleasedEvent lockReleasedEvent) { ``` ########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockAcquiredEvent.java: ########## @@ -0,0 +1,76 @@ +/* + * 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 org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** + * Event sent from TriggerManager operator to TableMaintenanceCoordinator to confirm that the lock + * has been acquired and the maintenance task is starting. + */ +public class LockAcquiredEvent implements OperatorEvent { + + private static final long serialVersionUID = 1L; + + private final boolean isRecoverLock; + + private final String lockId; + + public LockAcquiredEvent(boolean isRecoverLock, String lockId) { + this.isRecoverLock = isRecoverLock; Review Comment: This parameter (isRecoverLock) is only ever false and can thus be removed. -- 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]
