Guosmilesmile commented on code in PR #15151: URL: https://github.com/apache/iceberg/pull/15151#discussion_r2816069219
########## flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/BaseCoordinator.java: ########## @@ -0,0 +1,325 @@ +/* + * 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.CopyOnWriteArrayList; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +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.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; + protected static final Map<String, Consumer<LockReleaseEvent>> LOCK_RELEASE_CONSUMERS = + Maps.newConcurrentMap(); + protected static final List<LockReleaseEvent> PENDING_RELEASE_EVENTS = + new CopyOnWriteArrayList<>(); + + 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); + } + + @VisibleForTesting + void handleReleaseLock(LockReleaseEvent lockReleaseEvent) { Review Comment: Originally, `handleReleaseLock` was in `LRCoordinator`, but because a pending release lock needs to be handled once after `TMCoordinator` registers, I moved this method into the base class. For consistency and uniformity, I’d prefer to move both methods into the base class so the related logic is kept in one place. -- 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]
