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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/provider/WorkProvider.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.windmill.work.provider;
+
+import org.apache.beam.sdk.annotations.Internal;
+
+/**
+ * Provides {@link 
org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) for

Review Comment:
   It's a little odd that this interface itself doesn't relate at all how it 
provides work. For example with the current methods this could just be anything 
you coudl start and stop.  Do we think there will be more shared methods in the 
interface? Otherwise should we rename it to something just reflecting 
start/stop if that's all the usage needs.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/budget/HasGetWorkBudget.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.windmill.work.budget;
+
+/**
+ * Represents something that has/holds some {@link
+ * org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget}
+ */
+public interface HasGetWorkBudget {

Review Comment:
   nit: this name sounds more like a annotation/property than an interface to me
   
   GetWorkBudgetControlled?
   other ideas?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/provider/SingleSourceWorkProvider.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.windmill.work.provider;
+
+import com.google.auto.value.AutoValue;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.beam.runners.dataflow.worker.streaming.ComputationState;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream;
+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.WorkItemReceiver;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender;
+import org.apache.beam.sdk.annotations.Internal;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link WorkProvider} implementations that fetch {@link
+ * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) from 
a single source.
+ */
+@Internal
+public abstract class SingleSourceWorkProvider implements WorkProvider {
+  private static final Logger LOG = 
LoggerFactory.getLogger(SingleSourceWorkProvider.class);
+  protected final AtomicBoolean isRunning;
+  protected final WorkCommitter workCommitter;
+  protected final GetDataClient getDataClient;
+  protected final HeartbeatSender heartbeatSender;
+  protected final StreamingWorkScheduler streamingWorkScheduler;
+  protected final Runnable waitForResources;
+  protected final Function<String, Optional<ComputationState>> 
computationStateFetcher;
+  private final ExecutorService workProviderExecutor;
+
+  protected SingleSourceWorkProvider(
+      WorkCommitter workCommitter,
+      GetDataClient getDataClient,
+      HeartbeatSender heartbeatSender,
+      StreamingWorkScheduler streamingWorkScheduler,
+      Runnable waitForResources,
+      Function<String, Optional<ComputationState>> computationStateFetcher) {
+    this.workCommitter = workCommitter;
+    this.getDataClient = getDataClient;
+    this.heartbeatSender = heartbeatSender;
+    this.streamingWorkScheduler = streamingWorkScheduler;
+    this.waitForResources = waitForResources;
+    this.computationStateFetcher = computationStateFetcher;
+    this.workProviderExecutor =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setPriority(Thread.MIN_PRIORITY)
+                .setNameFormat(debugName() + "DispatchThread")
+                .build());
+    this.isRunning = new AtomicBoolean(false);
+  }
+
+  public static SingleSourceWorkProviderBuilder.Builder builder() {
+    return new 
AutoValue_SingleSourceWorkProvider_SingleSourceWorkProviderBuilder.Builder();
+  }
+
+  @SuppressWarnings("FutureReturnValueIgnored")
+  @Override
+  public final void start() {
+    if (isRunning.compareAndSet(false, true) && 
!workProviderExecutor.isShutdown()) {
+      workProviderExecutor.submit(
+          () -> {
+            LOG.info("Dispatch starting");
+            dispatchLoop();
+            LOG.info("Dispatch done");
+          });
+      workCommitter.start();

Review Comment:
   seems safer to start committer first, so it is ready if dispatch triggers a 
commit



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/provider/SingleSourceWorkProvider.java:
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.windmill.work.provider;
+
+import com.google.auto.value.AutoValue;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.function.Function;
+import org.apache.beam.runners.dataflow.worker.streaming.ComputationState;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream;
+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.WorkItemReceiver;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.StreamingWorkScheduler;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender;
+import org.apache.beam.sdk.annotations.Internal;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link WorkProvider} implementations that fetch {@link
+ * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) from 
a single source.
+ */
+@Internal
+public abstract class SingleSourceWorkProvider implements WorkProvider {
+  private static final Logger LOG = 
LoggerFactory.getLogger(SingleSourceWorkProvider.class);
+  protected final AtomicBoolean isRunning;
+  protected final WorkCommitter workCommitter;
+  protected final GetDataClient getDataClient;
+  protected final HeartbeatSender heartbeatSender;
+  protected final StreamingWorkScheduler streamingWorkScheduler;
+  protected final Runnable waitForResources;
+  protected final Function<String, Optional<ComputationState>> 
computationStateFetcher;
+  private final ExecutorService workProviderExecutor;
+
+  protected SingleSourceWorkProvider(
+      WorkCommitter workCommitter,
+      GetDataClient getDataClient,
+      HeartbeatSender heartbeatSender,
+      StreamingWorkScheduler streamingWorkScheduler,
+      Runnable waitForResources,
+      Function<String, Optional<ComputationState>> computationStateFetcher) {
+    this.workCommitter = workCommitter;
+    this.getDataClient = getDataClient;
+    this.heartbeatSender = heartbeatSender;
+    this.streamingWorkScheduler = streamingWorkScheduler;
+    this.waitForResources = waitForResources;
+    this.computationStateFetcher = computationStateFetcher;
+    this.workProviderExecutor =
+        Executors.newSingleThreadScheduledExecutor(
+            new ThreadFactoryBuilder()
+                .setDaemon(true)
+                .setPriority(Thread.MIN_PRIORITY)
+                .setNameFormat(debugName() + "DispatchThread")
+                .build());
+    this.isRunning = new AtomicBoolean(false);
+  }
+
+  public static SingleSourceWorkProviderBuilder.Builder builder() {
+    return new 
AutoValue_SingleSourceWorkProvider_SingleSourceWorkProviderBuilder.Builder();
+  }
+
+  @SuppressWarnings("FutureReturnValueIgnored")
+  @Override
+  public final void start() {
+    if (isRunning.compareAndSet(false, true) && 
!workProviderExecutor.isShutdown()) {
+      workProviderExecutor.submit(
+          () -> {
+            LOG.info("Dispatch starting");
+            dispatchLoop();
+            LOG.info("Dispatch done");
+          });
+      workCommitter.start();
+    }
+  }
+
+  @Override
+  public final void shutdown() {
+    if (isRunning.get() && !workProviderExecutor.isShutdown()) {
+      workProviderExecutor.shutdown();
+      boolean isTerminated = false;
+      try {
+        isTerminated = workProviderExecutor.awaitTermination(10, 
TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Unable to shutdown WorkProvider");
+      }
+
+      if (!isTerminated) {
+        workProviderExecutor.shutdownNow();
+      }
+      workCommitter.stop();
+      isRunning.set(false);
+    }
+  }
+
+  protected abstract void dispatchLoop();
+
+  protected abstract String debugName();
+
+  @AutoValue
+  public abstract static class SingleSourceWorkProviderBuilder {
+    public abstract WorkCommitter workCommitter();
+
+    public abstract GetDataClient getDataClient();
+
+    public abstract HeartbeatSender heartbeatSender();
+
+    public abstract StreamingWorkScheduler streamingWorkScheduler();
+
+    public abstract Runnable waitForResources();
+
+    public abstract Function<String, Optional<ComputationState>> 
computationStateFetcher();
+
+    @AutoValue.Builder
+    public abstract static class Builder {
+      public abstract Builder setWorkCommitter(WorkCommitter value);
+
+      public abstract Builder setGetDataClient(GetDataClient value);
+
+      public abstract Builder setHeartbeatSender(HeartbeatSender value);
+
+      public abstract Builder setStreamingWorkScheduler(StreamingWorkScheduler 
value);
+
+      public abstract Builder setWaitForResources(Runnable value);
+
+      public abstract Builder setComputationStateFetcher(
+          Function<String, Optional<ComputationState>> value);
+
+      abstract SingleSourceWorkProviderBuilder autoBuild();
+
+      public final WorkProvider build(Supplier<Windmill.GetWorkResponse> 
getWorkFn) {
+        SingleSourceWorkProviderBuilder params = autoBuild();
+        return new ApplianceWorkProvider(

Review Comment:
   the abstract base classes can be confusing since the contract between base 
and impls can be complex or hard to document.  I think you could perhaps just 
make this class non-abstract and have constructor start different dispatch 
loops with bound variables.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/getdata/FanOutWorkRefreshClient.java:
##########
@@ -44,7 +47,13 @@ public 
FanOutWorkRefreshClient(ThrottlingGetDataMetricTracker getDataMetricTrack
     this.getDataMetricTracker = getDataMetricTracker;
     this.fanOutActiveWorkRefreshExecutor =
         Executors.newCachedThreadPool(
-            new 
ThreadFactoryBuilder().setNameFormat(FAN_OUT_REFRESH_WORK_EXECUTOR_NAME).build());
+            new ThreadFactoryBuilder()
+                // FanOutWorkRefreshClient runs as a background process, don't 
let failures crash
+                // the worker.
+                .setUncaughtExceptionHandler(
+                    (t, e) -> LOG.error("Unexpected failure in {}", 
t.getName(), e))

Review Comment:
   What are the crashes? we probably prefer to crash the process than for 
example lose the active work refresher thread and the worker getting stuck 
doing continuous retries. It seems the sources of exceptions should be 
fixed/handled closer to where they occur



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