dmvk commented on a change in pull request #18626:
URL: https://github.com/apache/flink/pull/18626#discussion_r799290632



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/OnMainThreadJobManagerRunnerRegistry.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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 {

Review comment:
       I think this should have slightly different semantics (considering this 
class is meant as a safeguard) -> guarding all methods. If any of the read 
method would be accessed from a different thread, there is no memory safety / 
synchronization guarantee, so the results might not be up to date

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java
##########
@@ -129,115 +129,4 @@ public void tearDown() throws Exception {
     protected BlobServer getBlobServer() {
         return blobServer;
     }
-
-    /** A convenient builder for the {@link TestingDispatcher}. */
-    public class TestingDispatcherBuilder {

Review comment:
       👍 

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java
##########
@@ -169,35 +170,103 @@ public Dispatcher(
             DispatcherBootstrapFactory dispatcherBootstrapFactory,
             DispatcherServices dispatcherServices)
             throws Exception {
+        this(
+                rpcService,
+                fencingToken,
+                recoveredJobs,
+                recoveredDirtyJobs,
+                dispatcherBootstrapFactory,
+                dispatcherServices,
+                new DefaultJobManagerRunnerRegistry(16));
+    }
+
+    private Dispatcher(
+            RpcService rpcService,
+            DispatcherId fencingToken,
+            Collection<JobGraph> recoveredJobs,
+            Collection<JobResult> globallyTerminatedJobs,
+            DispatcherBootstrapFactory dispatcherBootstrapFactory,
+            DispatcherServices dispatcherServices,
+            JobManagerRunnerRegistry jobManagerRunnerRegistry)
+            throws Exception {
+        this(
+                rpcService,
+                fencingToken,
+                recoveredJobs,
+                globallyTerminatedJobs,
+                dispatcherServices.getConfiguration(),
+                dispatcherServices.getHighAvailabilityServices(),
+                dispatcherServices.getResourceManagerGatewayRetriever(),
+                dispatcherServices.getHeartbeatServices(),
+                dispatcherServices.getBlobServer(),
+                dispatcherServices.getFatalErrorHandler(),
+                dispatcherServices.getJobGraphWriter(),
+                dispatcherServices.getJobResultStore(),
+                dispatcherServices.getJobManagerMetricGroup(),
+                dispatcherServices.getMetricQueryServiceAddress(),
+                dispatcherServices.getIoExecutor(),
+                dispatcherServices.getHistoryServerArchivist(),
+                dispatcherServices.getArchivedExecutionGraphStore(),
+                dispatcherServices.getJobManagerRunnerFactory(),
+                dispatcherBootstrapFactory,
+                dispatcherServices.getOperationCaches(),
+                jobManagerRunnerRegistry,
+                new DispatcherResourceCleanerFactory(jobManagerRunnerRegistry, 
dispatcherServices));
+    }
+
+    @VisibleForTesting
+    protected Dispatcher(

Review comment:
       I still fail to see this simplify things.
   
   - We basically only need to construct dispatcher services once in 
`TestingDispatcherBuilder#build()`
   - We should add new fields `registry` + `cleaner factory` to the dispatcher 
services to be consistent with the rest

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -721,89 +714,12 @@ public void 
testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception {
         final JobResult jobResult =
                 
TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID());
         dispatcher =
-                new TestingDispatcherBuilder()
-                        .setInitialJobGraphs(Collections.singleton(jobGraph))
-                        .setDirtyJobResults(Collections.singleton(jobResult))
+                createTestingDispatcherBuilder()
+                        .withRecoveredJobs(Collections.singleton(jobGraph))
+                        
.withRecoveredDirtyJobs(Collections.singleton(jobResult))
                         .build();
     }
 
-    /** Tests that a failing {@link JobManagerRunner} will be properly cleaned 
up. */
-    @Test
-    public void testFailingJobManagerRunnerCleanup() throws Exception {

Review comment:
       nevermind, it has been moved... 🤦 

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java
##########
@@ -721,89 +714,12 @@ public void 
testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception {
         final JobResult jobResult =
                 
TestingJobResultStore.createSuccessfulJobResult(jobGraph.getJobID());
         dispatcher =
-                new TestingDispatcherBuilder()
-                        .setInitialJobGraphs(Collections.singleton(jobGraph))
-                        .setDirtyJobResults(Collections.singleton(jobResult))
+                createTestingDispatcherBuilder()
+                        .withRecoveredJobs(Collections.singleton(jobGraph))
+                        
.withRecoveredDirtyJobs(Collections.singleton(jobResult))
                         .build();
     }
 
-    /** Tests that a failing {@link JobManagerRunner} will be properly cleaned 
up. */
-    @Test
-    public void testFailingJobManagerRunnerCleanup() throws Exception {

Review comment:
       Why is this test not relevant anymore?

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.cleanup;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.Executors;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.Executor;
+
+/** {@code TestingResourceCleanerFactory} for adding custom {@link 
ResourceCleaner} creation. */
+public class TestingResourceCleanerFactory implements ResourceCleanerFactory {
+
+    private final Collection<LocallyCleanableResource> 
locallyCleanableResources =
+            new ArrayList<>();
+    private final Collection<GloballyCleanableResource> 
globallyCleanableResources =
+            new ArrayList<>();
+
+    private final Executor cleanupExecutor;
+
+    public TestingResourceCleanerFactory() {
+        this(Executors.directExecutor());
+    }
+
+    private TestingResourceCleanerFactory(Executor cleanupExecutor) {
+        this.cleanupExecutor = cleanupExecutor;
+    }
+
+    public TestingResourceCleanerFactory withLocallyCleanableResource(
+            LocallyCleanableResource locallyCleanableResource) {
+        this.locallyCleanableResources.add(locallyCleanableResource);
+
+        return this;
+    }
+
+    public TestingResourceCleanerFactory withGloballyCleanableResource(
+            GloballyCleanableResource globallyCleanableResource) {
+        this.globallyCleanableResources.add(globallyCleanableResource);
+
+        return this;
+    }
+
+    @Override
+    public ResourceCleaner createLocalResourceCleaner(
+            ComponentMainThreadExecutor mainThreadExecutor) {
+        return jobId -> {
+            mainThreadExecutor.assertRunningInMainThread();
+            Throwable t = null;
+            for (LocallyCleanableResource locallyCleanableResource : 
locallyCleanableResources) {
+                try {
+                    locallyCleanableResource.localCleanupAsync(jobId, 
cleanupExecutor).get();
+                } catch (Throwable throwable) {
+                    t = ExceptionUtils.firstOrSuppressed(throwable, t);
+                }
+            }
+
+            return t != null
+                    ? FutureUtils.completedExceptionally(t)
+                    : FutureUtils.completedVoidFuture();
+        };
+    }
+
+    @Override
+    public ResourceCleaner createGlobalResourceCleaner(
+            ComponentMainThreadExecutor mainThreadExecutor) {
+        return jobId -> {
+            mainThreadExecutor.assertRunningInMainThread();
+            Throwable t = null;
+            for (GloballyCleanableResource globallyCleanableResource : 
globallyCleanableResources) {
+                try {
+                    globallyCleanableResource.globalCleanupAsync(jobId, 
cleanupExecutor).get();
+                } catch (Throwable throwable) {
+                    t = ExceptionUtils.firstOrSuppressed(throwable, t);
+                }
+            }
+
+            return t != null
+                    ? FutureUtils.completedExceptionally(t)
+                    : FutureUtils.completedVoidFuture();
+        };
+    }

Review comment:
       We can easily get rid of the code duplication here
   ```suggestion
       @Override
       public ResourceCleaner createLocalResourceCleaner(
               ComponentMainThreadExecutor mainThreadExecutor) {
           return createResourceCleaner(
                   mainThreadExecutor,
                   locallyCleanableResources,
                   LocallyCleanableResource::localCleanupAsync);
       }
   
       @Override
       public ResourceCleaner createGlobalResourceCleaner(
               ComponentMainThreadExecutor mainThreadExecutor) {
           return createResourceCleaner(
                   mainThreadExecutor,
                   globallyCleanableResources,
                   GloballyCleanableResource::globalCleanupAsync);
       }
   
       private <T> ResourceCleaner createResourceCleaner(
               ComponentMainThreadExecutor mainThreadExecutor,
               Collection<T> resources,
               DefaultResourceCleaner.CleanupFn<T> cleanupFn) {
           return jobId -> {
               mainThreadExecutor.assertRunningInMainThread();
               Throwable t = null;
               for (T resource : resources) {
                   try {
                       cleanupFn.cleanupAsync(resource, jobId, 
cleanupExecutor).get();
                   } catch (Throwable throwable) {
                       t = ExceptionUtils.firstOrSuppressed(throwable, t);
                   }
               }
               return t != null
                       ? FutureUtils.completedExceptionally(t)
                       : FutureUtils.completedVoidFuture();
           };
       }
   ```

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java
##########
@@ -91,4 +165,212 @@ void completeJobExecution(ExecutionGraphInfo 
executionGraphInfo) {
     void waitUntilStarted() {
         startFuture.join();
     }
+
+    public static TestingDispatcher.Builder builder() {
+        return new Builder();
+    }
+
+    public static class Builder {

Review comment:
       What's the reason for changing all the prefixes from `set...` to 
`with...`?
   
   This will make any new tests to dispatcher harder to backport.

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java
##########
@@ -195,54 +142,71 @@ private TestingJobManagerRunnerFactory 
startDispatcherAndSubmitJob() throws Exce
 
     private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob(
             int numBlockingJobManagerRunners) throws Exception {
+        return startDispatcherAndSubmitJob(
+                createTestingDispatcherBuilder(), 
numBlockingJobManagerRunners);
+    }
+
+    private TestingJobManagerRunnerFactory startDispatcherAndSubmitJob(
+            TestingDispatcher.Builder dispatcherBuilder, int 
numBlockingJobManagerRunners)
+            throws Exception {
         final TestingJobManagerRunnerFactory testingJobManagerRunnerFactoryNG =
                 new 
TestingJobManagerRunnerFactory(numBlockingJobManagerRunners);
-        startDispatcher(testingJobManagerRunnerFactoryNG);
+        startDispatcher(dispatcherBuilder, testingJobManagerRunnerFactoryNG);
         submitJobAndWait();
 
         return testingJobManagerRunnerFactoryNG;
     }
 
     private void startDispatcher(JobManagerRunnerFactory 
jobManagerRunnerFactory) throws Exception {
-        TestingResourceManagerGateway resourceManagerGateway = new 
TestingResourceManagerGateway();
-        final HeartbeatServices heartbeatServices = new 
HeartbeatServices(1000L, 1000L);
-        final MemoryExecutionGraphInfoStore archivedExecutionGraphStore =
-                new MemoryExecutionGraphInfoStore();
-        dispatcher =
-                new TestingDispatcher(
-                        rpcService,
-                        DispatcherId.generate(),
-                        Collections.emptyList(),
-                        Collections.emptyList(),
-                        (dispatcher, scheduledExecutor, errorHandler) ->
-                                new NoOpDispatcherBootstrap(),
-                        new DispatcherServices(
-                                configuration,
-                                highAvailabilityServices,
-                                () -> 
CompletableFuture.completedFuture(resourceManagerGateway),
-                                blobServer,
-                                heartbeatServices,
-                                archivedExecutionGraphStore,
-                                
testingFatalErrorHandlerResource.getFatalErrorHandler(),
-                                VoidHistoryServerArchivist.INSTANCE,
-                                null,
-                                new DispatcherOperationCaches(),
-                                
UnregisteredMetricGroups.createUnregisteredJobManagerMetricGroup(),
-                                jobGraphWriter,
-                                jobResultStore,
-                                jobManagerRunnerFactory,
-                                ForkJoinPool.commonPool()));
+        startDispatcher(createTestingDispatcherBuilder(), 
jobManagerRunnerFactory);
+    }
+
+    private void startDispatcher(
+            TestingDispatcher.Builder dispatcherBuilder,
+            JobManagerRunnerFactory jobManagerRunnerFactory)
+            throws Exception {
+        dispatcher = 
dispatcherBuilder.withJobManagerRunnerFactory(jobManagerRunnerFactory).build();
 
         dispatcher.start();
 
         dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class);
     }
 
+    private TestingDispatcher.Builder createTestingDispatcherBuilder() {
+        final JobManagerRunnerRegistry jobManagerRunnerRegistry =
+                new DefaultJobManagerRunnerRegistry(2);
+        return TestingDispatcher.builder()
+                .withRpcService(rpcService)
+                .withBlobServer(blobServer)
+                .withJobManagerRunnerRegistry(jobManagerRunnerRegistry)
+                
.withFatalErrorHandler(testingFatalErrorHandlerResource.getFatalErrorHandler())
+                .withResourceCleanerFactory(
+                        new TestingResourceCleanerFactory()
+                                // JobManagerRunnerRegistry needs to be added 
explicitly
+                                // because cleaning it will trigger the 
closeAsync latch
+                                // provided by TestingJobManagerRunner
+                                
.withLocallyCleanableResource(jobManagerRunnerRegistry)
+                                .withGloballyCleanableResource(
+                                        (jobId, ignoredExecutor) -> {
+                                            
globalCleanupFuture.complete(jobId);
+                                            return 
FutureUtils.completedVoidFuture();
+                                        })
+                                .withLocallyCleanableResource(
+                                        (jobId, ignoredExecutor) -> {
+                                            localCleanupFuture.complete(jobId);
+                                            return 
FutureUtils.completedVoidFuture();
+                                        }));

Review comment:
       It looks like a builder, but it's not a builder 🤔 :)

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/TestingResourceCleanerFactory.java
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.cleanup;
+
+import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.concurrent.Executors;
+import org.apache.flink.util.concurrent.FutureUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.concurrent.Executor;
+
+/** {@code TestingResourceCleanerFactory} for adding custom {@link 
ResourceCleaner} creation. */
+public class TestingResourceCleanerFactory implements ResourceCleanerFactory {
+
+    private final Collection<LocallyCleanableResource> 
locallyCleanableResources =
+            new ArrayList<>();
+    private final Collection<GloballyCleanableResource> 
globallyCleanableResources =
+            new ArrayList<>();
+
+    private final Executor cleanupExecutor;
+
+    public TestingResourceCleanerFactory() {
+        this(Executors.directExecutor());
+    }
+
+    private TestingResourceCleanerFactory(Executor cleanupExecutor) {

Review comment:
       we should either get rid of this constructor or make it visible




-- 
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]


Reply via email to