scwhittle commented on code in PR #34148:
URL: https://github.com/apache/beam/pull/34148#discussion_r2081402485
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java:
##########
@@ -273,10 +282,14 @@ public synchronized void shutdown() {
}
private void consumeWorkerMetadata(WindmillEndpoints windmillEndpoints) {
+ LOG.info("DEBUG LOG: consumeWorkerMetadata called with endpoints: {}",
windmillEndpoints);
Review Comment:
rm
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -393,6 +408,41 @@ private StreamingDataflowWorker(
LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport());
}
+ private GetDataClient createGetDataClient(
+ DataflowWorkerHarnessOptions options,
+ WindmillServerStub windmillServer,
+ ThrottlingGetDataMetricTracker getDataMetricTracker,
+ @Nullable WindmillStreamPool<GetDataStream> getDataStreamPool) {
+ if (options.isEnableStreamingEngine()) {
+ Preconditions.checkNotNull(
+ getDataStreamPool, "getDataStreamPool must be initialized for
Streaming Engine");
+ return new StreamPoolGetDataClient(getDataMetricTracker,
getDataStreamPool);
+ } else {
+ return new ApplianceGetDataClient(windmillServer, getDataMetricTracker);
+ }
+ }
+
+ private WorkCommitter createWorkCommitter(
+ DataflowWorkerHarnessOptions options,
+ WindmillServerStub windmillServer,
+ int numCommitThreads,
+ Consumer<CompleteCommit> onCommitComplete) {
+ WeightedSemaphore<Commit> maxCommitByteSemaphore =
Commits.maxCommitByteSemaphore();
Review Comment:
agree,scope can be reduced
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamPoolSender.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * 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.harness;
+
+import static
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
+
+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.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import javax.annotation.concurrent.ThreadSafe;
+import org.apache.beam.runners.dataflow.worker.streaming.ComputationState;
+import org.apache.beam.runners.dataflow.worker.streaming.Watermarks;
+import org.apache.beam.runners.dataflow.worker.streaming.Work;
+import
org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest;
+import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection;
+import
org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetWorkStream;
+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.client.grpc.GrpcWindmillStreamFactory;
+import
org.apache.beam.runners.dataflow.worker.windmill.client.throttling.StreamingEngineThrottleTimers;
+import org.apache.beam.runners.dataflow.worker.windmill.work.WorkItemReceiver;
+import
org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget;
+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.Preconditions;
+import
org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Owns and maintains a pool of streams used to fetch {@link
+ * org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItem}(s) from
a specific source.
+ */
+@Internal
+@ThreadSafe
+public final class WindmillStreamPoolSender implements WindmillStreamSender {
+ private static final Logger LOG =
LoggerFactory.getLogger(WindmillStreamPoolSender.class);
+ private static final int GET_WORK_STREAM_TIMEOUT_MINUTES = 3;
+ private final AtomicReference<GetWorkBudget> getWorkBudget;
+ private final WindmillConnection connection;
+ private final GetWorkRequest getWorkRequest;
+ private final GrpcWindmillStreamFactory streamingEngineStreamFactory;
+ private final WorkCommitter workCommitter;
+ private final StreamingEngineThrottleTimers streamingEngineThrottleTimers;
+ private final GetDataClient getDataClient;
+ private final HeartbeatSender heartbeatSender;
+ private final StreamingWorkScheduler streamingWorkScheduler;
+ private final Runnable waitForResources;
+ private final Function<String, Optional<ComputationState>>
computationStateFetcher;
+ private final ExecutorService workProviderExecutor;
+ private final AtomicBoolean started;
+ private GetWorkStream getWorkStream;
+
+ private WindmillStreamPoolSender(
+ WindmillConnection connection,
+ GetWorkRequest getWorkRequest,
+ AtomicReference<GetWorkBudget> getWorkBudget,
+ GrpcWindmillStreamFactory streamingEngineStreamFactory,
+ WorkCommitter workCommitter,
+ GetDataClient getDataClient,
+ HeartbeatSender heartbeatSender,
+ StreamingWorkScheduler streamingWorkScheduler,
+ Runnable waitForResources,
+ Function<String, Optional<ComputationState>> computationStateFetcher) {
+ this.started = new AtomicBoolean(false);
+ this.connection = connection;
+ this.getWorkRequest = getWorkRequest;
+ this.getWorkBudget = getWorkBudget;
+ this.streamingEngineStreamFactory = streamingEngineStreamFactory;
+ this.streamingEngineThrottleTimers =
StreamingEngineThrottleTimers.create();
+ this.getDataClient = getDataClient;
+ this.heartbeatSender = heartbeatSender;
+ this.streamingWorkScheduler = streamingWorkScheduler;
+ this.waitForResources = waitForResources;
+ this.computationStateFetcher = computationStateFetcher;
+ this.workCommitter = workCommitter;
+ this.workProviderExecutor =
+ Executors.newSingleThreadExecutor(
+ new ThreadFactoryBuilder()
+ .setDaemon(true)
+ .setPriority(Thread.MIN_PRIORITY)
+ .setNameFormat("DispatchThread")
+ .build());
+ WorkItemReceiver workItemReceiver =
+ (computationId,
+ inputDataWatermark,
+ synchronizedProcessingTime,
+ workItem,
+ serializedWorkItemSize,
+ getWorkStreamLatencies) ->
+ this.computationStateFetcher
+ .apply(computationId)
+ .ifPresent(
+ computationState -> {
+ this.waitForResources.run();
+ this.streamingWorkScheduler.scheduleWork(
+ computationState,
+ workItem,
+ serializedWorkItemSize,
+ Watermarks.builder()
+
.setInputDataWatermark(Preconditions.checkNotNull(inputDataWatermark))
+
.setSynchronizedProcessingTime(synchronizedProcessingTime)
+
.setOutputDataWatermark(workItem.getOutputDataWatermark())
+ .build(),
+ Work.createProcessingContext(
+ computationId,
+ this.getDataClient,
+ workCommitter::commit,
+ this.heartbeatSender),
+ getWorkStreamLatencies);
+ });
+ this.getWorkStream =
Review Comment:
ping
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/GlobalDataStreamSender.java:
##########
@@ -58,6 +58,28 @@ public void close() {
delegate.shutdown();
}
+ @Override
+ public void start() {
+ throw new UnsupportedOperationException("start() not supported for
GlobalDataStreamSender");
+ }
+
+ @Override
+ public void setBudget(long items, long bytes) {
+ throw new UnsupportedOperationException("setBudget() not supported for
GlobalDataStreamSender");
Review Comment:
Can we rever the change to get rid of StreamSender? It seems better to have
the more restricted interface which is enforced at compile time than having a
bunch of unimplemented exception methods.
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java:
##########
@@ -424,23 +460,28 @@ private GlobalDataStreamSender getOrCreateGlobalDataSteam(
keyedEndpoint.getValue()));
}
- private WindmillStreamSender createAndStartWindmillStreamSender(Endpoint
endpoint) {
- WindmillStreamSender windmillStreamSender =
- WindmillStreamSender.create(
- WindmillConnection.from(endpoint, this::createWindmillStub),
- GetWorkRequest.newBuilder()
- .setClientId(jobHeader.getClientId())
- .setJobId(jobHeader.getJobId())
- .setProjectId(jobHeader.getProjectId())
- .setWorkerId(jobHeader.getWorkerId())
- .build(),
- GetWorkBudget.noBudget(),
- streamFactory,
- workItemScheduler,
- getDataStream ->
- StreamGetDataClient.create(
- getDataStream, this::getGlobalDataStream,
getDataMetricTracker),
- workCommitterFactory);
+ private WindmillStreamSender createAndStartWindmillStreamSender(
+ Endpoint endpoint, EndpointType enpointType) {
+ WindmillStreamSender windmillStreamSender;
+ windmillStreamSender =
Review Comment:
combine definition and initialization
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java:
##########
@@ -337,14 +363,15 @@ private CompletableFuture<Void>
closeStreamsNotIn(WindmillEndpoints newWindmillE
.map(
sender ->
CompletableFuture.runAsync(
- () -> closeStreamSender(sender.endpoint(), sender),
windmillStreamManager));
+ () -> closeStreamSender(sender.endpoint(),
(WindmillStreamSender) sender),
Review Comment:
I don't think you need the cast it is a subtype
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillEndpoints.java:
##########
@@ -70,11 +73,16 @@ public static WindmillEndpoints from(
endpointProto ->
Endpoint.from(endpointProto,
workerMetadataResponseProto.getExternalEndpoint()))
.collect(toImmutableSet());
-
+ // EndpointType endpointType =
Review Comment:
ping
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -243,17 +245,31 @@ private StreamingDataflowWorker(
@Nullable ChannelzServlet channelzServlet = null;
Consumer<PrintWriter> getDataStatusProvider;
Supplier<Long> currentActiveCommitBytesProvider;
-
- if (options.isEnableStreamingEngine() &&
options.getIsWindmillServiceDirectPathEnabled()) {
- // Direct path pipelines.
+ WindmillStreamPool<GetDataStream> getDataStreamPool =
Review Comment:
I don't think this is needed on the appliance path.
Maybe it could be structured as
```
if (appliance) {
} else {
common SE stuff
if (direct path) {
} else {
}
}
```
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java:
##########
@@ -393,6 +408,41 @@ private StreamingDataflowWorker(
LOG.debug("LocalWindmillHostport: {}", options.getLocalWindmillHostport());
}
+ private GetDataClient createGetDataClient(
Review Comment:
can these two helpers be static?
--
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]