xintongsong commented on a change in pull request #13004: URL: https://github.com/apache/flink/pull/13004#discussion_r468971000
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/resourcemanager/active/ResourceEventHandler.java ########## @@ -0,0 +1,60 @@ +/* + * 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.flink.runtime.resourcemanager.active; + +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.clusterframework.types.ResourceIDRetrievable; + +import java.util.Collection; + +/** + * Callback interfaces for handling resource events from external resource managers. + */ +public interface ResourceEventHandler<WorkerType extends ResourceIDRetrievable> { + + /** + * Notifies that workers of previous attempt have been recovered from the external resource manager. + * + * @param recoveredWorkers Collection of worker nodes, in the deployment specific type. + */ + void onPreviousAttemptWorkersRecovered(Collection<WorkerType> recoveredWorkers); + + /** + * Notifies that the worker has been terminated. + * + * <p>See also {@link ResourceManagerDriver#requestResource}. + * + * @param resourceId Identifier of the terminated worker. + */ + void onWorkerTerminated(ResourceID resourceId); + + /** + * Notifies that an error has occurred that the process cannot proceed. + * + * @param exception Exception that describes the error. + */ + void onError(Throwable exception); + + /** + * Execute given runnable in the rpc main thread. + * + * @param runnable Runnable to be executed. + */ + void handleInMainThread(Runnable runnable); Review comment: There's probably an easier way to keep the main thread executor on driver side up to date. We can introduce the following implementation of `Executor` as an inner class of `ActiveResourceManager`, and pass it into `ResourceManagerDriver.initialize()`. ``` /** * Always execute on the current main thread executor. */ private class GatewayExecutor implements Executor { @Override public void execute(Runnable command) { getMainThreadExecutor().execute(command); } } ``` This should be equivalent to currently calling `ResourceEventHandler.handleInMainThread()`, while achieving better separation of concerns. WDYT? ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
