scwhittle commented on code in PR #32922:
URL: https://github.com/apache/beam/pull/32922#discussion_r1816168706


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java:
##########
@@ -103,11 +106,22 @@ public void start() {
         "Multiple calls to {}.start() are not allowed.",
         getClass());
     workCommitter.start();
-    workProviderExecutor.execute(
-        () -> {
-          getDispatchLoop().run();
-          LOG.info("Dispatch done");
-        });
+    while (isRunning.get()) {
+      Future<?> dispatchLoopFuture =
+          workProviderExecutor.submit(
+              () -> {
+                getDispatchLoop().run();
+                LOG.info("Dispatch done");
+              });
+
+      try {
+        dispatchLoopFuture.get();

Review Comment:
   this is changing start() to be blocking.
   
   If we want that we shoudl have clearer method name and also just get rid of 
the workProviderExecutor and use this thread directly.



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarnessTest.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.beam.runners.dataflow.worker.streaming.harness;
+
+import static com.google.common.truth.Truth.assertThat;
+import static org.junit.Assert.assertThrows;
+import static org.mockito.Mockito.mock;
+
+import java.util.Optional;
+import java.util.function.Function;
+import org.apache.beam.runners.dataflow.worker.streaming.ComputationState;
+import 
org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter;
+import 
org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class SingleSourceWorkerHarnessTest {
+  private final WorkCommitter workCommitter = mock(WorkCommitter.class);
+  private final GetDataClient getDataClient = mock(GetDataClient.class);
+  private final HeartbeatSender heartbeatSender = mock(HeartbeatSender.class);
+  private final Runnable waitForResources = () -> {};
+  private final Function<String, Optional<ComputationState>> 
computationStateFetcher =
+      ignored -> Optional.empty();
+  private final StreamingWorkScheduler streamingWorkScheduler = 
mock(StreamingWorkScheduler.class);
+
+  private SingleSourceWorkerHarness createWorkerHarness(
+      SingleSourceWorkerHarness.GetWorkSender getWorkSender) {
+    return SingleSourceWorkerHarness.builder()
+        .setWorkCommitter(workCommitter)
+        .setGetDataClient(getDataClient)
+        .setHeartbeatSender(heartbeatSender)
+        .setWaitForResources(waitForResources)
+        .setStreamingWorkScheduler(streamingWorkScheduler)
+        .setComputationStateFetcher(computationStateFetcher)
+        .setGetWorkSender(getWorkSender)
+        .build();
+  }
+
+  @Test
+  public void testDispatchLoopFailureThrowsException_appliance() {
+    RuntimeException expected = new RuntimeException("something bad happened");
+    SingleSourceWorkerHarness.GetWorkSender getWorkSender =
+        SingleSourceWorkerHarness.GetWorkSender.forAppliance(
+            () -> {
+              throw expected;

Review Comment:
   I'm confused why the test of this isn't causing the runtime to halt.  It 
makes me think the unhandled exception handler isn't being called and this is 
just working because of the changes to make start() blocking and call get on 
the future().



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TerminatingExecutors.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.beam.runners.dataflow.worker.util;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadFactory;
+import org.apache.beam.runners.dataflow.worker.WorkerUncaughtExceptionHandler;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+
+/**
+ * Utility class for {@link java.util.concurrent.ExecutorService}s that will 
terminate the JVM on
+ * uncaught exceptions.
+ *
+ * @implNote Ensures that all threads produced by the {@link ExecutorService}s 
have a {@link
+ *     WorkerUncaughtExceptionHandler} attached to prevent hidden/silent 
exceptions and errors.
+ */
+public final class TerminatingExecutors {
+  private TerminatingExecutors() {}
+
+  public static ExecutorService newSingleThreadedExecutor(
+      ThreadFactoryBuilder threadFactoryBuilder, Logger logger) {
+    return Executors.newSingleThreadExecutor(
+        terminatingThreadFactory(threadFactoryBuilder, logger));
+  }
+
+  public static ScheduledExecutorService newSingleThreadedScheduledExecutor(
+      ThreadFactoryBuilder threadFactoryBuilder, Logger logger) {
+    return Executors.newSingleThreadScheduledExecutor(
+        terminatingThreadFactory(threadFactoryBuilder, logger));
+  }
+
+  public static ExecutorService newCachedThreadPool(
+      ThreadFactoryBuilder threadFactoryBuilder, Logger logger) {
+    return 
Executors.newCachedThreadPool(terminatingThreadFactory(threadFactoryBuilder, 
logger));
+  }
+
+  public static ExecutorService newFixedThreadPool(
+      int numThreads, ThreadFactoryBuilder threadFactoryBuilder, Logger 
logger) {
+    return Executors.newFixedThreadPool(
+        numThreads, terminatingThreadFactory(threadFactoryBuilder, logger));
+  }
+
+  private static ThreadFactory terminatingThreadFactory(
+      ThreadFactoryBuilder threadFactoryBuilder, Logger logger) {
+    return threadFactoryBuilder
+        .setUncaughtExceptionHandler(new 
WorkerUncaughtExceptionHandler(logger))

Review Comment:
   I'm not sure this is working if the executor thread is catchign the 
exception and putting it in the future.
   
   I think we might need instead to wrap the executor so that scheduled tasks 
are wrapped and then catch exceptions and call termination method like 
com/google/common/util/concurrent/WrappingExecutorService.java



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