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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ExecutionState.java:
##########
@@ -35,8 +42,13 @@ public abstract class ExecutionState {
 
   public abstract ExecutionStateTracker executionStateTracker();
 
-  public static ExecutionState.Builder builder() {
-    return new AutoValue_ExecutionState.Builder();
+  public final void close() {

Review Comment:
   should we have this state be autoclosable?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java:
##########
@@ -116,6 +118,11 @@ public boolean activateWork(ShardedKey shardedKey, Work 
work) {
     }
   }
 
+  public boolean activateWork(Work work) {

Review Comment:
   instead of overload can the other variant just be removed?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java:
##########
@@ -62,6 +63,7 @@ public ComputationState(
     this.transformUserNameToStateFamily = 
ImmutableMap.copyOf(transformUserNameToStateFamily);
     this.executionStateQueue = new ConcurrentLinkedQueue<>();
     this.activeWorkState = ActiveWorkState.create(computationStateCache);
+    this.sourceBytesProcessCounter = "dataflow_source_bytes_processed-" + 
mapTask.getSystemName();

Review Comment:
   how about sourceBytesProcessCounterName for var and method below?
   soudns like this would be a counter object from var name



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.processing;
+
+import com.google.api.services.dataflow.model.MapTask;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler;
+import org.apache.beam.runners.dataflow.worker.DataflowMapTaskExecutor;
+import org.apache.beam.runners.dataflow.worker.DataflowWorkExecutor;
+import org.apache.beam.runners.dataflow.worker.HotKeyLogger;
+import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException;
+import 
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.ComputationState;
+import org.apache.beam.runners.dataflow.worker.streaming.ExecutionState;
+import 
org.apache.beam.runners.dataflow.worker.streaming.KeyCommitTooLargeException;
+import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.streaming.Work;
+import 
org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters;
+import 
org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher;
+import 
org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter;
+import 
org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.client.commits.Commit;
+import 
org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.FailureTracker;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.WorkFailureProcessor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Schedules execution of user code to process a {@link
+ * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem} then 
commits the work item
+ * back to streaming execution backend.
+ */
+@Internal
+@ThreadSafe
+public final class StreamingWorkScheduler {
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamingWorkScheduler.class);
+
+  private final DataflowWorkerHarnessOptions options;
+  private final Supplier<Instant> clock;
+  private final ExecutionStateFactory executionStateFactory;
+  private final SideInputStateFetcher sideInputStateFetcher;
+  private final FailureTracker failureTracker;
+  private final WorkFailureProcessor workFailureProcessor;
+  private final StreamingCommitFinalizer commitFinalizer;
+  private final StreamingCounters streamingCounters;
+  private final HotKeyLogger hotKeyLogger;
+  private final ConcurrentMap<String, StageInfo> stageInfoMap;
+  private final DataflowExecutionStateSampler sampler;
+  private final AtomicInteger maxWorkItemCommitBytes;
+
+  public StreamingWorkScheduler(
+      DataflowWorkerHarnessOptions options,
+      Supplier<Instant> clock,
+      ExecutionStateFactory executionStateFactory,
+      SideInputStateFetcher sideInputStateFetcher,
+      FailureTracker failureTracker,
+      WorkFailureProcessor workFailureProcessor,
+      StreamingCommitFinalizer commitFinalizer,
+      StreamingCounters streamingCounters,
+      HotKeyLogger hotKeyLogger,
+      ConcurrentMap<String, StageInfo> stageInfoMap,
+      DataflowExecutionStateSampler sampler,
+      AtomicInteger maxWorkItemCommitBytes) {
+    this.options = options;
+    this.clock = clock;
+    this.executionStateFactory = executionStateFactory;
+    this.sideInputStateFetcher = sideInputStateFetcher;
+    this.failureTracker = failureTracker;
+    this.workFailureProcessor = workFailureProcessor;
+    this.commitFinalizer = commitFinalizer;
+    this.streamingCounters = streamingCounters;
+    this.hotKeyLogger = hotKeyLogger;
+    this.stageInfoMap = stageInfoMap;
+    this.sampler = sampler;
+    this.maxWorkItemCommitBytes = maxWorkItemCommitBytes;
+  }
+
+  private static ShardedKey createShardedKey(Work work) {

Review Comment:
   should Work contain a ShardedKey instead?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderCache.java:
##########
@@ -39,7 +39,7 @@
 @SuppressWarnings({
   "nullness" // TODO(https://github.com/apache/beam/issues/20497)
 })
-class ReaderCache {
+public class ReaderCache {

Review Comment:
   add internal annotation since now public



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkScheduler.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.processing;
+
+import com.google.api.services.dataflow.model.MapTask;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler;
+import org.apache.beam.runners.dataflow.worker.DataflowMapTaskExecutor;
+import org.apache.beam.runners.dataflow.worker.DataflowWorkExecutor;
+import org.apache.beam.runners.dataflow.worker.HotKeyLogger;
+import org.apache.beam.runners.dataflow.worker.WorkItemCancelledException;
+import 
org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
+import org.apache.beam.runners.dataflow.worker.streaming.ComputationState;
+import org.apache.beam.runners.dataflow.worker.streaming.ExecutionState;
+import 
org.apache.beam.runners.dataflow.worker.streaming.KeyCommitTooLargeException;
+import org.apache.beam.runners.dataflow.worker.streaming.ShardedKey;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import org.apache.beam.runners.dataflow.worker.streaming.Work;
+import 
org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters;
+import 
org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher;
+import 
org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter;
+import 
org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.client.commits.Commit;
+import 
org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.FailureTracker;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.WorkFailureProcessor;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.vendor.grpc.v1p60p1.com.google.protobuf.ByteString;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Schedules execution of user code to process a {@link
+ * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem} then 
commits the work item
+ * back to streaming execution backend.
+ */
+@Internal
+@ThreadSafe
+public final class StreamingWorkScheduler {
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamingWorkScheduler.class);
+
+  private final DataflowWorkerHarnessOptions options;
+  private final Supplier<Instant> clock;
+  private final ExecutionStateFactory executionStateFactory;
+  private final SideInputStateFetcher sideInputStateFetcher;
+  private final FailureTracker failureTracker;
+  private final WorkFailureProcessor workFailureProcessor;
+  private final StreamingCommitFinalizer commitFinalizer;
+  private final StreamingCounters streamingCounters;
+  private final HotKeyLogger hotKeyLogger;
+  private final ConcurrentMap<String, StageInfo> stageInfoMap;
+  private final DataflowExecutionStateSampler sampler;
+  private final AtomicInteger maxWorkItemCommitBytes;
+
+  public StreamingWorkScheduler(
+      DataflowWorkerHarnessOptions options,
+      Supplier<Instant> clock,
+      ExecutionStateFactory executionStateFactory,
+      SideInputStateFetcher sideInputStateFetcher,
+      FailureTracker failureTracker,
+      WorkFailureProcessor workFailureProcessor,
+      StreamingCommitFinalizer commitFinalizer,
+      StreamingCounters streamingCounters,
+      HotKeyLogger hotKeyLogger,
+      ConcurrentMap<String, StageInfo> stageInfoMap,
+      DataflowExecutionStateSampler sampler,
+      AtomicInteger maxWorkItemCommitBytes) {
+    this.options = options;
+    this.clock = clock;
+    this.executionStateFactory = executionStateFactory;
+    this.sideInputStateFetcher = sideInputStateFetcher;
+    this.failureTracker = failureTracker;
+    this.workFailureProcessor = workFailureProcessor;
+    this.commitFinalizer = commitFinalizer;
+    this.streamingCounters = streamingCounters;
+    this.hotKeyLogger = hotKeyLogger;
+    this.stageInfoMap = stageInfoMap;
+    this.sampler = sampler;
+    this.maxWorkItemCommitBytes = maxWorkItemCommitBytes;
+  }
+
+  private static ShardedKey createShardedKey(Work work) {
+    return ShardedKey.create(work.getWorkItem().getKey(), 
work.getWorkItem().getShardingKey());
+  }
+
+  private static long computeShuffleBytesRead(Windmill.WorkItem workItem) {
+    return workItem.getMessageBundlesList().stream()
+        .flatMap(bundle -> bundle.getMessagesList().stream())
+        .map(Windmill.Message::getSerializedSize)
+        .map(size -> (long) size)
+        .reduce(0L, Long::sum);
+  }
+
+  private static Windmill.WorkItemCommitRequest.Builder 
initializeOutputBuilder(
+      ByteString key, Windmill.WorkItem workItem) {
+    return Windmill.WorkItemCommitRequest.newBuilder()
+        .setKey(key)
+        .setShardingKey(workItem.getShardingKey())
+        .setWorkToken(workItem.getWorkToken())
+        .setCacheToken(workItem.getCacheToken());
+  }
+
+  private static Windmill.WorkItemCommitRequest buildWorkItemTruncationRequest(
+      ByteString key, Windmill.WorkItem workItem, int estimatedCommitSize) {
+    Windmill.WorkItemCommitRequest.Builder outputBuilder = 
initializeOutputBuilder(key, workItem);
+    outputBuilder.setExceedsMaxWorkItemCommitBytes(true);
+    outputBuilder.setEstimatedWorkItemCommitBytes(estimatedCommitSize);
+    return outputBuilder.build();
+  }
+
+  /** Sets the stage name and workId of the Thread executing the {@link Work} 
for logging. */
+  private static void setUpWorkLoggingContext(String workLatencyTrackingId, 
String computationId) {
+    DataflowWorkerLoggingMDC.setWorkId(workLatencyTrackingId);
+    DataflowWorkerLoggingMDC.setStageName(computationId);
+  }
+
+  private static String getShuffleTaskStepName(MapTask mapTask) {
+    // The MapTask instruction is ordered by dependencies, such that the first 
element is
+    // always going to be the shuffle task.
+    return mapTask.getInstructions().get(0).getName();
+  }
+
+  private static long computeSourceBytesProcessed(
+      DataflowWorkExecutor workExecutor, String sourceBytesCounterName) {
+    HashMap<String, ElementCounter> counters =
+        ((DataflowMapTaskExecutor) workExecutor)
+            .getReadOperation()
+            .receivers[0]
+            .getOutputCounters();
+
+    return Optional.ofNullable(counters.get(sourceBytesCounterName))
+        .map(counter -> ((OutputObjectAndByteCounter) 
counter).getByteCount().getAndReset())
+        .orElse(0L);
+  }
+
+  /** Resets logging context of the Thread executing the {@link Work} for 
logging. */
+  private void resetWorkLoggingContext(String workLatencyTrackingId) {
+    sampler.resetForWorkId(workLatencyTrackingId);
+    DataflowWorkerLoggingMDC.setWorkId(null);
+    DataflowWorkerLoggingMDC.setStageName(null);
+  }
+
+  /**
+   * Schedule work for execution. Work may be executed immediately, or queued 
and executed in the
+   * future. Only one work may be "active" (currently executing) per key at a 
time.
+   */
+  public void scheduleWork(
+      ComputationState computationState,
+      Windmill.WorkItem workItem,
+      Work.Watermarks watermarks,
+      Work.ProcessingContext.WithProcessWorkFn processingContext,
+      Consumer<Windmill.WorkItem> ackWorkItemQueued,

Review Comment:
   seems convoluted, the caller is passing in this function and it is called 
unconditionally at the end of this function so the caller could just call 
whatever it wanted after this method returned instead of passing in this 
consumer



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java:
##########
@@ -164,23 +177,27 @@ public static StreamingEngineClient create(
       JobHeader jobHeader,
       GetWorkBudget totalGetWorkBudget,
       GrpcWindmillStreamFactory streamingEngineStreamFactory,
-      WorkItemProcessor processWorkItem,
+      WorkItemScheduler processWorkItem,
       ChannelCachingStubFactory channelCachingStubFactory,
       GetWorkBudgetDistributor getWorkBudgetDistributor,
-      GrpcDispatcherClient dispatcherClient) {
-    StreamingEngineClient streamingEngineClient =
-        new StreamingEngineClient(
-            jobHeader,
-            totalGetWorkBudget,
-            new AtomicReference<>(StreamingEngineConnectionState.EMPTY),
-            streamingEngineStreamFactory,
-            processWorkItem,
-            channelCachingStubFactory,
-            getWorkBudgetDistributor,
-            dispatcherClient,
-            new Random().nextLong());
-    streamingEngineClient.start();
-    return streamingEngineClient;
+      GrpcDispatcherClient dispatcherClient,
+      Function<WindmillStream.CommitWorkStream, WorkCommitter> 
workCommitterFactory,
+      Consumer<List<Windmill.ComputationHeartbeatResponse>> heartbeatProcessor,
+      AtomicReference<StreamingEngineConnectionState> 
streamingEngineConnectionsState) {
+    // Set the connectionState to a valid state if it is null.
+    streamingEngineConnectionsState.compareAndSet(null, 
StreamingEngineConnectionState.EMPTY);
+    return new StreamingEngineClient(
+        jobHeader,
+        totalGetWorkBudget,
+        streamingEngineConnectionsState,
+        streamingEngineStreamFactory,
+        processWorkItem,
+        channelCachingStubFactory,
+        getWorkBudgetDistributor,
+        dispatcherClient,
+        new Random().nextLong(),

Review Comment:
   nit: add param comment or use a variable to document what this is.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java:
##########
@@ -226,8 +278,7 @@ private void startWorkerMetadataConsumer() {
         });
   }
 
-  @VisibleForTesting
-  void finish() {
+  public void finish() {
     if (!started.compareAndSet(true, false)) {

Review Comment:
   similarly could make it an error if not started
   and if we don't need to support starting/stopping/restarting don't set back 
to false



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkSchedulerFactory.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.processing;
+
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler;
+import org.apache.beam.runners.dataflow.worker.DataflowMapTaskExecutorFactory;
+import org.apache.beam.runners.dataflow.worker.HotKeyLogger;
+import org.apache.beam.runners.dataflow.worker.ReaderCache;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import 
org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters;
+import 
org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import 
org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.FailureTracker;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.WorkFailureProcessor;
+import org.apache.beam.sdk.fn.IdGenerator;
+import org.joda.time.Instant;
+
+public final class StreamingWorkSchedulerFactory {
+  private StreamingWorkSchedulerFactory() {}
+
+  public static StreamingWorkScheduler createStreamingWorkScheduler(
+      DataflowWorkerHarnessOptions options,
+      Supplier<Instant> clock,
+      ReaderCache readerCache,
+      DataflowMapTaskExecutorFactory mapTaskExecutorFactory,
+      BoundedQueueExecutor workExecutor,
+      Function<String, WindmillStateCache.ForComputation> stateCacheFactory,
+      Function<Windmill.GlobalDataRequest, Windmill.GlobalData> 
fetchGlobalDataFn,
+      FailureTracker failureTracker,
+      WorkFailureProcessor workFailureProcessor,
+      StreamingCounters streamingCounters,
+      HotKeyLogger hotKeyLogger,
+      DataflowExecutionStateSampler sampler,
+      AtomicInteger maxWorkItemCommitBytes,
+      IdGenerator idGenerator,
+      ConcurrentMap<String, StageInfo> stageInfoMap) {
+    StreamingCommitFinalizer streamingCommitFinalizer =
+        StreamingCommitFinalizer.create(workExecutor);
+    ExecutionStateFactory executionStateFactory =
+        new ExecutionStateFactory(
+            options,
+            mapTaskExecutorFactory,
+            readerCache,
+            stateCacheFactory,
+            sampler,
+            streamingCounters.pendingDeltaCounters(),
+            idGenerator);
+    SideInputStateFetcher sideInputStateFetcher =
+        new SideInputStateFetcher(fetchGlobalDataFn, options);
+
+    return new StreamingWorkScheduler(

Review Comment:
   Do we need the factory? How will this be different for the direct path? IT 
seems that the committer, reader fetch fn etc are injected elsewhere.
   
   If not just have a create method for the streamingworkscheduler.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java:
##########
@@ -164,23 +177,27 @@ public static StreamingEngineClient create(
       JobHeader jobHeader,
       GetWorkBudget totalGetWorkBudget,
       GrpcWindmillStreamFactory streamingEngineStreamFactory,
-      WorkItemProcessor processWorkItem,
+      WorkItemScheduler processWorkItem,
       ChannelCachingStubFactory channelCachingStubFactory,
       GetWorkBudgetDistributor getWorkBudgetDistributor,
-      GrpcDispatcherClient dispatcherClient) {
-    StreamingEngineClient streamingEngineClient =
-        new StreamingEngineClient(
-            jobHeader,
-            totalGetWorkBudget,
-            new AtomicReference<>(StreamingEngineConnectionState.EMPTY),
-            streamingEngineStreamFactory,
-            processWorkItem,
-            channelCachingStubFactory,
-            getWorkBudgetDistributor,
-            dispatcherClient,
-            new Random().nextLong());
-    streamingEngineClient.start();
-    return streamingEngineClient;
+      GrpcDispatcherClient dispatcherClient,
+      Function<WindmillStream.CommitWorkStream, WorkCommitter> 
workCommitterFactory,
+      Consumer<List<Windmill.ComputationHeartbeatResponse>> heartbeatProcessor,
+      AtomicReference<StreamingEngineConnectionState> 
streamingEngineConnectionsState) {

Review Comment:
   Can we remove the connections state injection? It is internal to this class 
except if other things (tests?) are going to peek at the atomic value.  Instead 
of injecting and then peeking at the internal state, it seems clearer if those 
users instead had a visiblefortesting method they called.
   



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java:
##########
@@ -204,15 +223,48 @@ static StreamingEngineClient forTesting(
             stubFactory,
             getWorkBudgetDistributor,
             dispatcherClient,
-            clientId);
+            clientId,
+            workCommitterFactory,
+            heartbeatResponseProcessor);
     streamingEngineClient.start();
     return streamingEngineClient;
   }
 
-  private void start() {
-    startGetWorkerMetadataStream();
-    startWorkerMetadataConsumer();
-    getWorkBudgetRefresher.start();
+  public void start() {
+    if (started.compareAndSet(false, true)) {

Review Comment:
   could just make it an error to start twice. 
   Otherwise it is kind of racy since the second starter doesn't wait for the 
first start to finish and thus might observe things in odd state



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ExecutionState.java:
##########
@@ -23,9 +23,16 @@
 import org.apache.beam.runners.dataflow.worker.DataflowWorkExecutor;
 import org.apache.beam.runners.dataflow.worker.StreamingModeExecutionContext;
 import org.apache.beam.sdk.coders.Coder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @AutoValue
 public abstract class ExecutionState {

Review Comment:
   annotate internal



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/StreamingWorkSchedulerFactory.java:
##########
@@ -0,0 +1,87 @@
+/*
+ * 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.processing;
+
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.DataflowExecutionStateSampler;
+import org.apache.beam.runners.dataflow.worker.DataflowMapTaskExecutorFactory;
+import org.apache.beam.runners.dataflow.worker.HotKeyLogger;
+import org.apache.beam.runners.dataflow.worker.ReaderCache;
+import org.apache.beam.runners.dataflow.worker.streaming.StageInfo;
+import 
org.apache.beam.runners.dataflow.worker.streaming.harness.StreamingCounters;
+import 
org.apache.beam.runners.dataflow.worker.streaming.sideinput.SideInputStateFetcher;
+import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import 
org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.FailureTracker;
+import 
org.apache.beam.runners.dataflow.worker.windmill.work.processing.failures.WorkFailureProcessor;
+import org.apache.beam.sdk.fn.IdGenerator;
+import org.joda.time.Instant;
+
+public final class StreamingWorkSchedulerFactory {
+  private StreamingWorkSchedulerFactory() {}
+
+  public static StreamingWorkScheduler createStreamingWorkScheduler(

Review Comment:
   perhaps this should take in the StreamingCommitFinalizer, 
ExecutionStateFactory and SideIdStateInputFetcher instead of a bunch of params 
it then forwards to those?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java:
##########
@@ -276,8 +327,14 @@ public ImmutableList<Long> getAndResetThrottleTimes() {
 
     return ImmutableList.<Long>builder()

Review Comment:
   don't create the immutable list if reducing anyway



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/StreamingEngineClient.java:
##########
@@ -164,23 +177,27 @@ public static StreamingEngineClient create(
       JobHeader jobHeader,
       GetWorkBudget totalGetWorkBudget,
       GrpcWindmillStreamFactory streamingEngineStreamFactory,
-      WorkItemProcessor processWorkItem,
+      WorkItemScheduler processWorkItem,
       ChannelCachingStubFactory channelCachingStubFactory,
       GetWorkBudgetDistributor getWorkBudgetDistributor,
-      GrpcDispatcherClient dispatcherClient) {
-    StreamingEngineClient streamingEngineClient =
-        new StreamingEngineClient(
-            jobHeader,
-            totalGetWorkBudget,
-            new AtomicReference<>(StreamingEngineConnectionState.EMPTY),
-            streamingEngineStreamFactory,
-            processWorkItem,
-            channelCachingStubFactory,
-            getWorkBudgetDistributor,
-            dispatcherClient,
-            new Random().nextLong());
-    streamingEngineClient.start();
-    return streamingEngineClient;
+      GrpcDispatcherClient dispatcherClient,
+      Function<WindmillStream.CommitWorkStream, WorkCommitter> 
workCommitterFactory,
+      Consumer<List<Windmill.ComputationHeartbeatResponse>> heartbeatProcessor,
+      AtomicReference<StreamingEngineConnectionState> 
streamingEngineConnectionsState) {
+    // Set the connectionState to a valid state if it is null.
+    streamingEngineConnectionsState.compareAndSet(null, 
StreamingEngineConnectionState.EMPTY);
+    return new StreamingEngineClient(

Review Comment:
   start was removed from this method but is still done for testing method
   
   if intended, add to comment that start must be called.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDispatcherClient.java:
##########
@@ -101,11 +104,19 @@ ImmutableSet<HostAndPort> getDispatcherEndpoints() {
     return dispatcherStubs.get().dispatcherEndpoints();
   }
 
-  CloudWindmillMetadataServiceV1Alpha1Stub getWindmillMetadataServiceStub() {
+  /** Will block the calling thread until the initial endpoints are present. */
+  CloudWindmillMetadataServiceV1Alpha1Stub 
getWindmillMetadataServiceStubBlocking() {
+    try {
+      onInitializedEndpoints.await();

Review Comment:
   Log if it is going to block? to help with debugging in case this gets stuck.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java:
##########
@@ -99,18 +101,18 @@ private GrpcDirectGetWorkStream(
       int logEveryNStreamFailures,
       ThrottleTimer getWorkThrottleTimer,
       Supplier<GetDataStream> getDataStream,
-      Supplier<CommitWorkStream> commitWorkStream,
-      WorkItemProcessor workItemProcessorFn) {
+      Supplier<WorkCommitter> workCommitter,
+      WorkItemScheduler workItemSchedulerFn) {

Review Comment:
   nit: drop Fn from param name and member variable since it's a class not only 
a function



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcDirectGetWorkStream.java:
##########
@@ -316,5 +315,20 @@ private void runAndReset() {
       workTimingInfosTracker.reset();
       data = ByteString.EMPTY;
     }
+
+    private Work.ProcessingContext.WithProcessWorkFn 
createProcessingContext(String computationId) {
+      return Work.createProcessingContext(computationId, 
getDataStream.get()::requestKeyedData)
+          .setWorkCommitter(workCommitter.get()::commit)
+          .setGetDataStream(getDataStream.get());
+    }
+
+    private Work.Watermarks createWatermarks(WorkItem workItem) {

Review Comment:
   nit: how about making this static and passing in metadata? it makes it 
clearer when reading above that it uses metadatda for the watermarks not just 
hte work item.
   
   can also have a single check that metadata is set in runAndReset



-- 
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: github-unsubscr...@beam.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to