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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -174,7 +174,7 @@ private StreamingDataflowWorker(
       StreamingCounters streamingCounters,
       MemoryMonitor memoryMonitor,
       GrpcWindmillStreamFactory windmillStreamFactory,
-      Function<String, ScheduledExecutorService> executorSupplier,
+      Function<String, ScheduledExecutorService> activeWorkRefreshExecutorFn,

Review Comment:
   does this need to be a parameter still? can we just create a scheduled 
executor to pass into ActiveWorkRefresher?



##########
runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarnessTest.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.assertTrue;
+import static org.mockito.Mockito.mock;
+
+import java.util.Optional;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Function;
+import org.apache.beam.runners.dataflow.worker.WorkerUncaughtExceptionHandler;
+import org.apache.beam.runners.dataflow.worker.streaming.ComputationState;
+import org.apache.beam.runners.dataflow.worker.util.common.worker.JvmRuntime;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@RunWith(JUnit4.class)
+public class SingleSourceWorkerHarnessTest {
+  private static final Logger LOG = 
LoggerFactory.getLogger(SingleSourceWorkerHarnessTest.class);
+  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, JvmRuntime 
runtime) {
+    Thread.setDefaultUncaughtExceptionHandler(new 
WorkerUncaughtExceptionHandler(runtime, LOG));

Review Comment:
   // In non-test scenario this is set in ...



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -346,10 +348,7 @@ public static StreamingDataflowWorker 
fromOptions(DataflowWorkerHarnessOptions o
             .setSizeMb(options.getWorkerCacheMb())
             .setSupportMapViaMultimap(options.isEnableStreamingEngine())
             .build();
-    Function<String, ScheduledExecutorService> executorSupplier =
-        threadName ->
-            Executors.newSingleThreadScheduledExecutor(

Review Comment:
   yeah, since that schedules at a fixed rate internally to ActiveWorkRefresher 
I don think we want to crash the vm.



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