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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -526,60 +535,72 @@ public static StreamingDataflowWorker 
fromOptions(DataflowWorkerHarnessOptions o
           WorkUnitClient dataflowServiceClient,
           GrpcWindmillStreamFactory.Builder windmillStreamFactoryBuilder,
           Function<ComputationConfig.Fetcher, ComputationStateCache> 
computationStateCacheFactory) {
-    ComputationConfig.Fetcher configFetcher;
-    WindmillServerStub windmillServer;
-    ComputationStateCache computationStateCache;
-    GrpcWindmillStreamFactory windmillStreamFactory;
-    ConfigFetcherComputationStateCacheAndWindmillClient.Builder builder =
-        ConfigFetcherComputationStateCacheAndWindmillClient.builder();
     if (options.isEnableStreamingEngine()) {
       GrpcDispatcherClient dispatcherClient =
           GrpcDispatcherClient.create(options, new 
WindmillStubFactoryFactoryImpl(options));
-      configFetcher =
+      ComputationConfig.Fetcher configFetcher =
           StreamingEngineComputationConfigFetcher.create(
               options.getGlobalConfigRefreshPeriod().getMillis(), 
dataflowServiceClient);
       
configFetcher.getGlobalConfigHandle().registerConfigObserver(dispatcherClient::onJobConfig);
-      computationStateCache = 
computationStateCacheFactory.apply(configFetcher);
-      windmillStreamFactory =
+      ComputationStateCache computationStateCache =
+          computationStateCacheFactory.apply(configFetcher);
+      GrpcWindmillStreamFactory windmillStreamFactory =
           windmillStreamFactoryBuilder
               .setProcessHeartbeatResponses(
                   new 
WorkHeartbeatResponseProcessor(computationStateCache::get))
               .setHealthCheckIntervalMillis(
                   options.getWindmillServiceStreamingRpcHealthCheckPeriodMs())
               .build();
-      windmillServer = GrpcWindmillServer.create(options, 
windmillStreamFactory, dispatcherClient);
-      builder.setWindmillDispatcherClient(dispatcherClient);
+      return ConfigFetcherComputationStateCacheAndWindmillClient.builder()
+          .setWindmillDispatcherClient(dispatcherClient)
+          .setConfigFetcher(configFetcher)
+          .setComputationStateCache(computationStateCache)
+          .setWindmillStreamFactory(windmillStreamFactory)
+          .setWindmillServer(
+              GrpcWindmillServer.create(options, windmillStreamFactory, 
dispatcherClient))
+          .build();
     } else {

Review Comment:
   can remove else to reduce nesting since above always returns



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/StreamingWorkerHarness.java:
##########
@@ -25,4 +25,6 @@ public interface StreamingWorkerHarness {
   void start();
 
   void shutdown();
+
+  long getAndResetThrottleTime();

Review Comment:
   ditto could make StreamingWorkerHarness implement the 
ThrottlingTimerInterface (or whatever you name it).



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/SingleSourceWorkerHarness.java:
##########
@@ -74,7 +75,8 @@ public final class SingleSourceWorkerHarness implements 
StreamingWorkerHarness {
       StreamingWorkScheduler streamingWorkScheduler,
       Runnable waitForResources,
       Function<String, Optional<ComputationState>> computationStateFetcher,
-      GetWorkSender getWorkSender) {
+      GetWorkSender getWorkSender,
+      Supplier<Long> throttleTimeSupplier) {

Review Comment:
   can we make a functional interface instead of a supplier? It's not clear 
that the supplier should get+reset, generally Supplier seems like it would just 
get.
   
   Seems like this could then take supplier to vend but also could implement 
the same interface



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