scwhittle commented on code in PR #31133:
URL: https://github.com/apache/beam/pull/31133#discussion_r1584811492
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -387,24 +393,53 @@ private StreamingDataflowWorker(
LOG.debug("WindmillServiceEndpoint: {}",
options.getWindmillServiceEndpoint());
LOG.debug("WindmillServicePort: {}", options.getWindmillServicePort());
LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport());
- LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes);
+ LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes.get());
}
public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions options) {
long clientId = clientIdGenerator.nextLong();
MemoryMonitor memoryMonitor = MemoryMonitor.fromOptions(options);
- ConcurrentMap<String, ComputationState> computationMap = new
ConcurrentHashMap<>();
ConcurrentMap<String, StageInfo> stageInfo = new ConcurrentHashMap<>();
+ ConcurrentMap<String, String> stateNameMap = new ConcurrentHashMap<>();
StreamingCounters streamingCounters = StreamingCounters.create();
-
+ WorkUnitClient dataflowServiceClient = new DataflowWorkUnitClient(options,
LOG);
+ BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options);
+ AtomicInteger maxWorkItemCommitBytes = new
AtomicInteger(Integer.MAX_VALUE);
+ WindmillStateCache windmillStateCache =
+ WindmillStateCache.ofSizeMbs(options.getWorkerCacheMb());
+ Function<String, ScheduledExecutorService> executorSupplier =
+ threadName ->
+ Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder().setNameFormat(threadName).build());
GrpcWindmillStreamFactory windmillStreamFactory =
createWindmillStreamFactory(options, clientId);
- WindmillServerStub windmillServer =
- createWindmillServerStub(
- options,
- windmillStreamFactory,
- new WorkHeartbeatResponseProcessor(
- computationId ->
Optional.ofNullable(computationMap.get(computationId))));
+ WindmillServerStub windmillServer = createWindmillServerStub(options,
windmillStreamFactory);
+ ComputationConfig.Fetcher configFetcher =
+ options.isEnableStreamingEngine()
+ ? StreamingEngineConfigFetcher.forTesting(
Review Comment:
forTesting isn't a good name if this is in the regular path
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/config/StreamingPipelineConfig.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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 com.google.api.services.dataflow.model.StreamingComputationConfig;
+import com.google.auto.value.AutoValue;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Optional;
+import javax.annotation.Nullable;
+import org.apache.beam.sdk.annotations.Internal;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort;
+
+/** Pipeline configuration for jobs running w/ Streaming Engine. */
+@AutoValue
+@Internal
+public abstract class StreamingPipelineConfig {
+
+ private static final long DEFAULT_MAX_WORK_ITEM_COMMIT_BYTES = 180 << 20;
+
+ public static StreamingPipelineConfig.Builder builder() {
+ return new AutoValue_StreamingPipelineConfig.Builder()
+ .setMaxWorkItemCommitBytes(DEFAULT_MAX_WORK_ITEM_COMMIT_BYTES)
+ .setComputationConfig(null)
+ .setUserStepToStateFamilyNameMap(new HashMap<>())
+ .setWindmillServiceEndpoints(ImmutableSet.of());
+ }
+
+ public static StreamingPipelineConfig forAppliance(
Review Comment:
can we just get rid of this and have the appliance config use builder() and
set these?
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -387,24 +393,53 @@ private StreamingDataflowWorker(
LOG.debug("WindmillServiceEndpoint: {}",
options.getWindmillServiceEndpoint());
LOG.debug("WindmillServicePort: {}", options.getWindmillServicePort());
LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport());
- LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes);
+ LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes.get());
}
public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions options) {
long clientId = clientIdGenerator.nextLong();
MemoryMonitor memoryMonitor = MemoryMonitor.fromOptions(options);
- ConcurrentMap<String, ComputationState> computationMap = new
ConcurrentHashMap<>();
ConcurrentMap<String, StageInfo> stageInfo = new ConcurrentHashMap<>();
+ ConcurrentMap<String, String> stateNameMap = new ConcurrentHashMap<>();
StreamingCounters streamingCounters = StreamingCounters.create();
-
+ WorkUnitClient dataflowServiceClient = new DataflowWorkUnitClient(options,
LOG);
+ BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options);
+ AtomicInteger maxWorkItemCommitBytes = new
AtomicInteger(Integer.MAX_VALUE);
+ WindmillStateCache windmillStateCache =
+ WindmillStateCache.ofSizeMbs(options.getWorkerCacheMb());
+ Function<String, ScheduledExecutorService> executorSupplier =
+ threadName ->
+ Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder().setNameFormat(threadName).build());
GrpcWindmillStreamFactory windmillStreamFactory =
createWindmillStreamFactory(options, clientId);
- WindmillServerStub windmillServer =
- createWindmillServerStub(
- options,
- windmillStreamFactory,
- new WorkHeartbeatResponseProcessor(
- computationId ->
Optional.ofNullable(computationMap.get(computationId))));
+ WindmillServerStub windmillServer = createWindmillServerStub(options,
windmillStreamFactory);
+ ComputationConfig.Fetcher configFetcher =
+ options.isEnableStreamingEngine()
+ ? StreamingEngineConfigFetcher.forTesting(
+ true,
Review Comment:
rm param if hard-coded
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -373,10 +350,39 @@ private StreamingDataflowWorker(
clock,
options.getActiveWorkRefreshPeriodMillis(),
stuckCommitDurationMillis,
- () -> Collections.unmodifiableCollection(computationMap.values()),
+ computationStateCache::getAllComputations,
sampler,
metricTrackingWindmillServer::refreshActiveWork,
executorSupplier.apply("RefreshWork"));
+
+ WorkerStatusPages workerStatusPages =
+ WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor, () ->
true);
Review Comment:
can this last parameter be made optional via overload? or at least add a
comment here
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -387,24 +393,53 @@ private StreamingDataflowWorker(
LOG.debug("WindmillServiceEndpoint: {}",
options.getWindmillServiceEndpoint());
LOG.debug("WindmillServicePort: {}", options.getWindmillServicePort());
LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport());
- LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes);
+ LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes.get());
}
public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions options) {
long clientId = clientIdGenerator.nextLong();
MemoryMonitor memoryMonitor = MemoryMonitor.fromOptions(options);
- ConcurrentMap<String, ComputationState> computationMap = new
ConcurrentHashMap<>();
ConcurrentMap<String, StageInfo> stageInfo = new ConcurrentHashMap<>();
+ ConcurrentMap<String, String> stateNameMap = new ConcurrentHashMap<>();
StreamingCounters streamingCounters = StreamingCounters.create();
-
+ WorkUnitClient dataflowServiceClient = new DataflowWorkUnitClient(options,
LOG);
+ BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options);
+ AtomicInteger maxWorkItemCommitBytes = new
AtomicInteger(Integer.MAX_VALUE);
+ WindmillStateCache windmillStateCache =
+ WindmillStateCache.ofSizeMbs(options.getWorkerCacheMb());
+ Function<String, ScheduledExecutorService> executorSupplier =
+ threadName ->
+ Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder().setNameFormat(threadName).build());
GrpcWindmillStreamFactory windmillStreamFactory =
createWindmillStreamFactory(options, clientId);
- WindmillServerStub windmillServer =
- createWindmillServerStub(
- options,
- windmillStreamFactory,
- new WorkHeartbeatResponseProcessor(
- computationId ->
Optional.ofNullable(computationMap.get(computationId))));
+ WindmillServerStub windmillServer = createWindmillServerStub(options,
windmillStreamFactory);
+ ComputationConfig.Fetcher configFetcher =
+ options.isEnableStreamingEngine()
+ ? StreamingEngineConfigFetcher.forTesting(
+ true,
+ options.getGlobalConfigRefreshPeriod().getMillis(),
+ dataflowServiceClient,
+ executorSupplier,
+ FIX_MULTI_OUTPUT_INFOS_ON_PAR_DO_INSTRUCTIONS,
+ config ->
+ onPipelineConfig(
+ config,
+ stateNameMap,
+ windmillServer::setWindmillServiceEndpoints,
+ maxWorkItemCommitBytes))
+ : new StreamingApplianceConfigFetcher(
+ windmillServer,
+ config -> consumeUserStepToStateFamilyName(config,
stateNameMap),
+ FIX_MULTI_OUTPUT_INFOS_ON_PAR_DO_INSTRUCTIONS);
+ ComputationStateCache computationStateCache =
+ ComputationStateCache.create(
+ configFetcher, workExecutor, windmillStateCache::forComputation);
+ if (windmillServer instanceof GrpcWindmillServer) {
Review Comment:
this cast and lazy initialization is kind of gross
one idea could be to remove the windmillserver from the appliance config
fetcher, it could just construct it's own channel and sync stub directly.
and then have a separate start method on the configfetcher taking the
function to consume responses. Then you can make sure to call that after
everything is initialized.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -387,24 +393,53 @@ private StreamingDataflowWorker(
LOG.debug("WindmillServiceEndpoint: {}",
options.getWindmillServiceEndpoint());
LOG.debug("WindmillServicePort: {}", options.getWindmillServicePort());
LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport());
- LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes);
+ LOG.debug("maxWorkItemCommitBytes: {}", maxWorkItemCommitBytes.get());
}
public static StreamingDataflowWorker
fromOptions(DataflowWorkerHarnessOptions options) {
long clientId = clientIdGenerator.nextLong();
MemoryMonitor memoryMonitor = MemoryMonitor.fromOptions(options);
- ConcurrentMap<String, ComputationState> computationMap = new
ConcurrentHashMap<>();
ConcurrentMap<String, StageInfo> stageInfo = new ConcurrentHashMap<>();
+ ConcurrentMap<String, String> stateNameMap = new ConcurrentHashMap<>();
StreamingCounters streamingCounters = StreamingCounters.create();
-
+ WorkUnitClient dataflowServiceClient = new DataflowWorkUnitClient(options,
LOG);
+ BoundedQueueExecutor workExecutor = createWorkUnitExecutor(options);
+ AtomicInteger maxWorkItemCommitBytes = new
AtomicInteger(Integer.MAX_VALUE);
+ WindmillStateCache windmillStateCache =
+ WindmillStateCache.ofSizeMbs(options.getWorkerCacheMb());
+ Function<String, ScheduledExecutorService> executorSupplier =
+ threadName ->
+ Executors.newSingleThreadScheduledExecutor(
+ new ThreadFactoryBuilder().setNameFormat(threadName).build());
GrpcWindmillStreamFactory windmillStreamFactory =
createWindmillStreamFactory(options, clientId);
- WindmillServerStub windmillServer =
- createWindmillServerStub(
- options,
- windmillStreamFactory,
- new WorkHeartbeatResponseProcessor(
- computationId ->
Optional.ofNullable(computationMap.get(computationId))));
+ WindmillServerStub windmillServer = createWindmillServerStub(options,
windmillStreamFactory);
+ ComputationConfig.Fetcher configFetcher =
+ options.isEnableStreamingEngine()
+ ? StreamingEngineConfigFetcher.forTesting(
+ true,
+ options.getGlobalConfigRefreshPeriod().getMillis(),
+ dataflowServiceClient,
+ executorSupplier,
+ FIX_MULTI_OUTPUT_INFOS_ON_PAR_DO_INSTRUCTIONS,
+ config ->
+ onPipelineConfig(
+ config,
+ stateNameMap,
+ windmillServer::setWindmillServiceEndpoints,
+ maxWorkItemCommitBytes))
+ : new StreamingApplianceConfigFetcher(
+ windmillServer,
+ config -> consumeUserStepToStateFamilyName(config,
stateNameMap),
Review Comment:
seems like the consuming the config method could be the same for both SE and
appliance, where unset fields are just ignored
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -373,10 +350,39 @@ private StreamingDataflowWorker(
clock,
options.getActiveWorkRefreshPeriodMillis(),
stuckCommitDurationMillis,
- () -> Collections.unmodifiableCollection(computationMap.values()),
+ computationStateCache::getAllComputations,
sampler,
metricTrackingWindmillServer::refreshActiveWork,
executorSupplier.apply("RefreshWork"));
+
+ WorkerStatusPages workerStatusPages =
+ WorkerStatusPages.create(DEFAULT_STATUS_PORT, memoryMonitor, () ->
true);
+ this.statusPages =
+ windmillServiceEnabled
+ ? StreamingWorkerStatusPages.forStreamingEngine(
Review Comment:
would a builder pattern be better than the separate forStreamingEngine and
forAppliance factory methods?
There seem to be a lot of common things.
--
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]