Guosmilesmile commented on code in PR #15042: URL: https://github.com/apache/iceberg/pull/15042#discussion_r2688782657
########## 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(); Review Comment: > If we keep both blocks, then we can just create a CoordinatorBasedLockManager and use the LockManager in the new TriggerManagerOperator. Means we can reuse `org.apache.iceberg.util. LockManager` to refactor the logic here? > Alternatively we use an absolutely different locking API. The CoordinatorBaseLock is able to send more info than a simple lock. We could just send back "Trigger with timestamp X is finished" info to the TriggerManagerOperator. In this case we can simplify the TriggerManagerOperator greatly as based on the finished timestamp it could handle the recovery and the normal trigger in the same way. For now ,in my design `TriggerManagerOperator` and `LockRemoverOperator` don't share the same coordinator , they share the same lock in coordinator use static. So `LockRemoverOperator` send finished timestamp to `TriggerManagerOperator` not supported at the moment,but we can share this information by setting the finished timestamp as a static variable. I’ve gone through it again. Is the logic here basically to merge `lock` and `recoveryLock` into a single timestamped lock, and then use the timestamp to determine whether the corresponding lock has finished? If that’s the case, there will be a situation during restore where the trigger with timestamp 1 is still running, and we need to emit a trigger with timestamp 2 to replace `recoverLock`. Once 2 is finished, we clean up all locks with timestamps less than 2. Is this what I’ve understood correctly? As for option 1 or 2, I don’t have a preference for now. Both approaches require changes, and the outcome is unknown, so we need to try them out. -- 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]
