dmvk commented on a change in pull request #18626: URL: https://github.com/apache/flink/pull/18626#discussion_r799467460
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/OnMainThreadJobManagerRunnerRegistry.java ########## @@ -0,0 +1,109 @@ +/* + * 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.dispatcher; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor; +import org.apache.flink.runtime.jobmaster.JobManagerRunner; + +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Executor; + +/** + * {@code OnMainThreadJobManagerRunnerRegistry} implements {@link JobManagerRunnerRegistry} guarding + * the passed {@code JobManagerRunnerRegistry} instance in a way that it only allows modifying + * methods to be executed on the component's main thread. + * + * @see ComponentMainThreadExecutor + */ +public class OnMainThreadJobManagerRunnerRegistry implements JobManagerRunnerRegistry { + + private final JobManagerRunnerRegistry delegate; + private final ComponentMainThreadExecutor mainThreadExecutor; + + public OnMainThreadJobManagerRunnerRegistry( + JobManagerRunnerRegistry delegate, ComponentMainThreadExecutor mainThreadExecutor) { + this.delegate = delegate; + this.mainThreadExecutor = mainThreadExecutor; + } + + @Override + public boolean isRegistered(JobID jobId) { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.isRegistered(jobId); + } + + @Override + public void register(JobManagerRunner jobManagerRunner) { + mainThreadExecutor.assertRunningInMainThread(); + delegate.register(jobManagerRunner); + } + + @Override + public JobManagerRunner get(JobID jobId) { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.get(jobId); + } + + @Override + public int size() { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.size(); + } + + @Override + public Set<JobID> getRunningJobIds() { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.getRunningJobIds(); + } + + @Override + public Collection<JobManagerRunner> getJobManagerRunners() { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.getJobManagerRunners(); + } + + @Override + public CompletableFuture<Void> globalCleanupAsync(JobID jobId, Executor executor) { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.globalCleanupAsync(jobId, executor); + } + + @Override + public CompletableFuture<Void> localCleanupAsync(JobID jobId, Executor executor) { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.localCleanupAsync(jobId, executor); + } + + @Override + public JobManagerRunner unregister(JobID jobId) { + mainThreadExecutor.assertRunningInMainThread(); + return delegate.unregister(jobId); + } + + /** + * Returns the delegated {@link JobManagerRunnerRegistry}. This method can be used to workaround + * the main thread safeguard. + */ + public JobManagerRunnerRegistry getDelegate() { Review comment: We have a WrappingProxy interface for this already ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java ########## @@ -1119,7 +1158,11 @@ private void jobMasterFailed(JobID jobId, Throwable cause) { private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) { jobManagerMetricGroup.gauge( - MetricNames.NUM_RUNNING_JOBS, () -> (long) jobManagerRunnerRegistry.size()); + MetricNames.NUM_RUNNING_JOBS, + // metrics can be called from anywhere and therefore, have to run without the main + // thread safeguard being triggered. For metrics, we can afford to be not 100% + // accurate Review comment: I'm not sure about this note. @zentol can you confirm that there is no synchronization for metric querying? -- 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]
