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


##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -718,100 +834,27 @@ int numCommitThreads() {
   public void start() {
     running.set(true);
 
-    if (windmillServiceEnabled) {
-      // Schedule the background getConfig thread. Blocks until windmillServer 
stub is ready.
-      schedulePeriodicGlobalConfigRequests();
-    }
+    configFetcher.start();
 
     memoryMonitorThread.start();
     dispatchThread.start();
     sampler.start();
 
-    if (options.getPeriodicStatusPageOutputDirectory() != null) {
-      ScheduledExecutorService statusPageTimer = 
executorSupplier.apply("DumpStatusPages");
-      statusPageTimer.scheduleWithFixedDelay(
-          () -> {
-            Collection<Capturable> pages = statusPages.getDebugCapturePages();
-            if (pages.isEmpty()) {
-              LOG.warn("No captured status pages.");
-            }
-            long timestamp = clock.get().getMillis();
-            for (Capturable page : pages) {
-              PrintWriter writer = null;
-              try {
-                File outputFile =
-                    new File(
-                        options.getPeriodicStatusPageOutputDirectory(),
-                        ("StreamingDataflowWorker"
-                                + options.getWorkerId()
-                                + "_"
-                                + page.pageName()
-                                + timestamp
-                                + ".html")
-                            .replaceAll("/", "_"));
-                writer = new PrintWriter(outputFile, UTF_8.name());
-                page.captureData(writer);
-              } catch (IOException e) {
-                LOG.warn("Error dumping status page.", e);
-              } finally {
-                if (writer != null) {
-                  writer.close();
-                }
-              }
-            }
-          },
-          60,
-          60,
-          TimeUnit.SECONDS);
-      scheduledExecutors.add(statusPageTimer);
-    }
     workCommitter.start();
     workerStatusReporter.start();
     activeWorkRefresher.start();
   }
 
   public void startStatusPages() {

Review Comment:
   can this be inlined into start() and removed?



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -471,10 +548,38 @@ static StreamingDataflowWorker forTesting(
       Function<String, ScheduledExecutorService> executorSupplier,
       int localRetryTimeoutMs) {
     ConcurrentMap<String, StageInfo> stageInfo = new ConcurrentHashMap<>();
+    ConcurrentMap<String, String> stateNameMap = new ConcurrentHashMap<>();
+    AtomicInteger maxWorkItemCommitBytes = new 
AtomicInteger(Integer.MAX_VALUE);
     BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options);
     WindmillStateCache stateCache = 
WindmillStateCache.ofSizeMbs(options.getWorkerCacheMb());
-    computationMap.putAll(
-        createComputationMapForTesting(mapTasks, workExecutor, 
stateCache::forComputation));
+    ComputationConfig.Fetcher configFetcher =
+        options.isEnableStreamingEngine()
+            ? StreamingEngineConfigFetcher.forTesting(
+                /* hasReceivedGlobalConfig= */ true,
+                options.getGlobalConfigRefreshPeriod().getMillis(),
+                workUnitClient,
+                executorSupplier,
+                FIX_MULTI_OUTPUT_INFOS_ON_PAR_DO_INSTRUCTIONS,
+                config ->
+                    onPipelineConfig(
+                        config,
+                        stateNameMap,
+                        windmillServer::setWindmillServiceEndpoints,
+                        maxWorkItemCommitBytes))
+            : new StreamingApplianceConfigFetcher(
+                windmillServer,
+                config ->
+                    onPipelineConfig(config, stateNameMap, ignored -> {}, 
maxWorkItemCommitBytes),
+                FIX_MULTI_OUTPUT_INFOS_ON_PAR_DO_INSTRUCTIONS);
+    ComputationStateCache computationStateCache =
+        ComputationStateCache.create(configFetcher, workExecutor, 
stateCache::forComputation);
+    computationStateCache.loadCacheForTesting(
+        mapTasks.stream()
+            .map(FIX_MULTI_OUTPUT_INFOS_ON_PAR_DO_INSTRUCTIONS)
+            .collect(Collectors.toList()),
+        workExecutor,
+        stateCache::forComputation);
+    computationStateCacheRef.set(computationStateCache);

Review Comment:
   how about a VisibleForTesting annotated method to access the 
ComputationStateCache instead?  Seems convoluted to pass in atomic ref to 
assign to.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -931,15 +941,19 @@ void streamingDispatchLoop() {
                   Instant inputDataWatermark,
                   Instant synchronizedProcessingTime,
                   Windmill.WorkItem workItem,
-                  Collection<LatencyAttribution> getWorkStreamLatencies) -> {
-                memoryMonitor.waitForResources("GetWork");
-                scheduleWorkItem(
-                    getComputationState(computation),
-                    inputDataWatermark,
-                    synchronizedProcessingTime,
-                    workItem,
-                    getWorkStreamLatencies);
-              });
+                  Collection<LatencyAttribution> getWorkStreamLatencies) ->
+                  computationStateCache
+                      .get(computation)
+                      .ifPresent(

Review Comment:
   what happened before on missing computation? In either case it seems like we 
should throw exception or log error as otherwise we're just dropping the work 
item silently which will be confusing.



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceConfigFetcher.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.config;
+
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap;
+
+import com.google.api.services.dataflow.model.MapTask;
+import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import 
org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigResponse;
+import 
org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigResponse.NameMapEntry;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.gcp.util.Transport;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Table;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Fetches computation config from Streaming Appliance. */
+@Internal
+@ThreadSafe
+public final class StreamingApplianceConfigFetcher implements 
ComputationConfig.Fetcher {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamingApplianceConfigFetcher.class);
+
+  private final WindmillServerStub windmillServer;
+  private final Consumer<StreamingPipelineConfig> onPipelineConfig;
+  private final ConcurrentHashMap<String, String> systemNameToComputationIdMap;
+  private final Function<MapTask, MapTask> fixMapTaskMultiOutputInfoFn;
+
+  public StreamingApplianceConfigFetcher(
+      WindmillServerStub windmillServer,
+      Consumer<StreamingPipelineConfig> onPipelineConfig,
+      Function<MapTask, MapTask> fixMapTaskMultiOutputInfoFn) {
+    this.windmillServer = windmillServer;
+    this.onPipelineConfig = onPipelineConfig;

Review Comment:
   seems odd to have this listener hidden in the fetcher
   
   Maybe instead the listener should be on whatever is driving the fetching. I 
think that woudl be the cache.
   
   Additionally I think that the FIX_MULTI_OUTPUT_INFOS_ON_PAR_DO_INSTRUCTIONS 
could move into the cache where fetches are being performed.  As is it is 
injected into all teh fetchers and applied to the testing generated configs.  
That would let you just do it once and possibly coudl remove from 
StreamingDataflowWorker to the cache



##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingApplianceConfigFetcher.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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.config;
+
+import static 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap;
+
+import com.google.api.services.dataflow.model.MapTask;
+import java.io.IOException;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Consumer;
+import java.util.function.Function;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import 
org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigResponse;
+import 
org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetConfigResponse.NameMapEntry;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.extensions.gcp.util.Transport;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables;
+import 
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Table;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Fetches computation config from Streaming Appliance. */
+@Internal
+@ThreadSafe
+public final class StreamingApplianceConfigFetcher implements 
ComputationConfig.Fetcher {

Review Comment:
   should this be StreamingApplianceComputationConfigFetcher? ditto with SE



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