m-trieu commented on code in PR #31902:
URL: https://github.com/apache/beam/pull/31902#discussion_r1757365595
##########
runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/FanOutStreamingEngineWorkerHarness.java:
##########
@@ -272,42 +263,106 @@ private void startWorkerMetadataConsumer() {
public synchronized void shutdown() {
Preconditions.checkState(started, "StreamingEngineClient never started.");
getWorkerMetadataStream.get().halfClose();
- getWorkBudgetRefresher.stop();
newWorkerMetadataPublisher.shutdownNow();
newWorkerMetadataConsumer.shutdownNow();
channelCachingStubFactory.shutdown();
}
+ private void consumeWindmillWorkerEndpoints(WindmillEndpoints
newWindmillEndpoints) {
+ consumeEndpoints(newWindmillEndpoints).join();
+ }
+
/**
* {@link java.util.function.Consumer<WindmillEndpoints>} used to update
{@link #connections} on
* new backend worker metadata.
*/
- private synchronized void consumeWindmillWorkerEndpoints(WindmillEndpoints
newWindmillEndpoints) {
- isBudgetRefreshPaused.set(true);
+ private synchronized CompletableFuture<Void> consumeEndpoints(
+ WindmillEndpoints newWindmillEndpoints) {
LOG.info("Consuming new windmill endpoints: {}", newWindmillEndpoints);
ImmutableMap<Endpoint, WindmillConnection> newWindmillConnections =
createNewWindmillConnections(newWindmillEndpoints.windmillEndpoints());
-
+ CompletableFuture<Void> closeStaleStreams =
+ closeStaleStreams(newWindmillConnections.values(),
connections.get().windmillStreams());
+ ImmutableMap<WindmillConnection, WindmillStreamSender> newStreams =
+ createAndStartNewStreams(newWindmillConnections.values()).join();
StreamingEngineConnectionState newConnectionsState =
StreamingEngineConnectionState.builder()
.setWindmillConnections(newWindmillConnections)
- .setWindmillStreams(
-
closeStaleStreamsAndCreateNewStreams(newWindmillConnections.values()))
+ .setWindmillStreams(newStreams)
.setGlobalDataStreams(
createNewGlobalDataStreams(newWindmillEndpoints.globalDataEndpoints()))
.build();
-
LOG.info(
"Setting new connections: {}. Previous connections: {}.",
newConnectionsState,
connections.get());
connections.set(newConnectionsState);
- isBudgetRefreshPaused.set(false);
- getWorkBudgetRefresher.requestBudgetRefresh();
+ getWorkBudgetDistributor.distributeBudget(newStreams.values(),
totalGetWorkBudget);
+
+ // Close the streams outside the lock.
Review Comment:
i was thinking, do we want to block the consumer thread here? or just let
the future complete and not block future metadata based on the streams getting
closed
--
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]