Skip to content
/ beam Public
forked from apache/beam

Commit

Permalink
add shutdown and start mechanics to windmill streams (apache#32774)
Browse files Browse the repository at this point in the history
* simplify budget distribution logic moving it into GrpcDirectGetWorkStream, simplify worker metadata consumption
* add unit tests for GrpcGetDataStreamRequests utilities, moved converting QueuedBatch into StreamingGetDataReequest into QueueBatch
* don't block on rendering status pages
  • Loading branch information
m-trieu authored Nov 19, 2024
1 parent 2ed33b5 commit 5229c93
Show file tree
Hide file tree
Showing 44 changed files with 3,278 additions and 739 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,12 @@ public WorkItemCancelledException(long sharding_key) {
super("Work item cancelled for key " + sharding_key);
}

public WorkItemCancelledException(Throwable e) {
super(e);
public WorkItemCancelledException(String message, Throwable cause) {
super(message, cause);
}

public WorkItemCancelledException(Throwable cause) {
super(cause);
}

/** Returns whether an exception was caused by a {@link WorkItemCancelledException}. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap.toImmutableMap;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet.toImmutableSet;

import java.io.Closeable;
import com.google.errorprone.annotations.CanIgnoreReturnValue;
import java.util.HashSet;
import java.util.Map.Entry;
import java.util.NoSuchElementException;
Expand All @@ -34,6 +34,7 @@
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import javax.annotation.CheckReturnValue;
import javax.annotation.Nullable;
import javax.annotation.concurrent.GuardedBy;
Expand Down Expand Up @@ -65,6 +66,7 @@
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.ImmutableMap;
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.collect.Streams;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.slf4j.Logger;
Expand Down Expand Up @@ -112,7 +114,7 @@ public final class FanOutStreamingEngineWorkerHarness implements StreamingWorker
private boolean started;

@GuardedBy("this")
private @Nullable GetWorkerMetadataStream getWorkerMetadataStream;
private @Nullable GetWorkerMetadataStream getWorkerMetadataStream = null;

private FanOutStreamingEngineWorkerHarness(
JobHeader jobHeader,
Expand Down Expand Up @@ -143,7 +145,6 @@ private FanOutStreamingEngineWorkerHarness(
this.totalGetWorkBudget = totalGetWorkBudget;
this.activeMetadataVersion = Long.MIN_VALUE;
this.workCommitterFactory = workCommitterFactory;
this.getWorkerMetadataStream = null;
}

/**
Expand Down Expand Up @@ -204,9 +205,10 @@ public synchronized void start() {
Preconditions.checkState(!started, "FanOutStreamingEngineWorkerHarness cannot start twice.");
getWorkerMetadataStream =
streamFactory.createGetWorkerMetadataStream(
dispatcherClient.getWindmillMetadataServiceStubBlocking(),
dispatcherClient::getWindmillMetadataServiceStubBlocking,
getWorkerMetadataThrottleTimer,
this::consumeWorkerMetadata);
getWorkerMetadataStream.start();
started = true;
}

Expand All @@ -225,7 +227,7 @@ public ImmutableSet<HostAndPort> currentWindmillEndpoints() {
*/
private GetDataStream getGlobalDataStream(String globalDataKey) {
return Optional.ofNullable(backends.get().globalDataStreams().get(globalDataKey))
.map(GlobalDataStreamSender::get)
.map(GlobalDataStreamSender::stream)
.orElseThrow(
() -> new NoSuchElementException("No endpoint for global data tag: " + globalDataKey));
}
Expand All @@ -236,7 +238,8 @@ public synchronized void shutdown() {
Preconditions.checkState(started, "FanOutStreamingEngineWorkerHarness never started.");
Preconditions.checkNotNull(getWorkerMetadataStream).shutdown();
workerMetadataConsumer.shutdownNow();
closeStreamsNotIn(WindmillEndpoints.none());
// Close all the streams blocking until this completes to not leak resources.
closeStreamsNotIn(WindmillEndpoints.none()).join();
channelCachingStubFactory.shutdown();

try {
Expand Down Expand Up @@ -300,27 +303,38 @@ private synchronized void consumeWindmillWorkerEndpoints(WindmillEndpoints newWi
}

/** Close the streams that are no longer valid asynchronously. */
private void closeStreamsNotIn(WindmillEndpoints newWindmillEndpoints) {
@CanIgnoreReturnValue
private CompletableFuture<Void> closeStreamsNotIn(WindmillEndpoints newWindmillEndpoints) {
StreamingEngineBackends currentBackends = backends.get();
currentBackends.windmillStreams().entrySet().stream()
.filter(
connectionAndStream ->
!newWindmillEndpoints.windmillEndpoints().contains(connectionAndStream.getKey()))
.forEach(
entry ->
windmillStreamManager.execute(
() -> closeStreamSender(entry.getKey(), entry.getValue())));
Stream<CompletableFuture<Void>> closeStreamFutures =
currentBackends.windmillStreams().entrySet().stream()
.filter(
connectionAndStream ->
!newWindmillEndpoints
.windmillEndpoints()
.contains(connectionAndStream.getKey()))
.map(
entry ->
CompletableFuture.runAsync(
() -> closeStreamSender(entry.getKey(), entry.getValue()),
windmillStreamManager));

Set<Endpoint> newGlobalDataEndpoints =
new HashSet<>(newWindmillEndpoints.globalDataEndpoints().values());
currentBackends.globalDataStreams().values().stream()
.filter(sender -> !newGlobalDataEndpoints.contains(sender.endpoint()))
.forEach(
sender ->
windmillStreamManager.execute(() -> closeStreamSender(sender.endpoint(), sender)));
Stream<CompletableFuture<Void>> closeGlobalDataStreamFutures =
currentBackends.globalDataStreams().values().stream()
.filter(sender -> !newGlobalDataEndpoints.contains(sender.endpoint()))
.map(
sender ->
CompletableFuture.runAsync(
() -> closeStreamSender(sender.endpoint(), sender), windmillStreamManager));

return CompletableFuture.allOf(
Streams.concat(closeStreamFutures, closeGlobalDataStreamFutures)
.toArray(CompletableFuture[]::new));
}

private void closeStreamSender(Endpoint endpoint, Closeable sender) {
private void closeStreamSender(Endpoint endpoint, StreamSender sender) {
LOG.debug("Closing streams to endpoint={}, sender={}", endpoint, sender);
try {
sender.close();
Expand All @@ -346,13 +360,14 @@ private void closeStreamSender(Endpoint endpoint, Closeable sender) {
private CompletionStage<Pair<Endpoint, WindmillStreamSender>>
getOrCreateWindmillStreamSenderFuture(
Endpoint endpoint, ImmutableMap<Endpoint, WindmillStreamSender> currentStreams) {
return MoreFutures.supplyAsync(
() ->
Pair.of(
endpoint,
Optional.ofNullable(currentStreams.get(endpoint))
.orElseGet(() -> createAndStartWindmillStreamSender(endpoint))),
windmillStreamManager);
return Optional.ofNullable(currentStreams.get(endpoint))
.map(backend -> CompletableFuture.completedFuture(Pair.of(endpoint, backend)))
.orElseGet(
() ->
MoreFutures.supplyAsync(
() -> Pair.of(endpoint, createAndStartWindmillStreamSender(endpoint)),
windmillStreamManager)
.toCompletableFuture());
}

/** Add up all the throttle times of all streams including GetWorkerMetadataStream. */
Expand Down Expand Up @@ -393,9 +408,8 @@ private GlobalDataStreamSender getOrCreateGlobalDataSteam(
.orElseGet(
() ->
new GlobalDataStreamSender(
() ->
streamFactory.createGetDataStream(
createWindmillStub(keyedEndpoint.getValue()), new ThrottleTimer()),
streamFactory.createGetDataStream(
createWindmillStub(keyedEndpoint.getValue()), new ThrottleTimer()),
keyedEndpoint.getValue()));
}

Expand All @@ -416,7 +430,7 @@ private WindmillStreamSender createAndStartWindmillStreamSender(Endpoint endpoin
StreamGetDataClient.create(
getDataStream, this::getGlobalDataStream, getDataMetricTracker),
workCommitterFactory);
windmillStreamSender.startStreams();
windmillStreamSender.start();
return windmillStreamSender;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,44 +17,45 @@
*/
package org.apache.beam.runners.dataflow.worker.streaming.harness;

import java.io.Closeable;
import java.util.function.Supplier;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillEndpoints.Endpoint;
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream.GetDataStream;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers;

@Internal
@ThreadSafe
// TODO (m-trieu): replace Supplier<Stream> with Stream after github.com/apache/beam/pull/32774/ is
// merged
final class GlobalDataStreamSender implements Closeable, Supplier<GetDataStream> {
final class GlobalDataStreamSender implements StreamSender {
private final Endpoint endpoint;
private final Supplier<GetDataStream> delegate;
private final GetDataStream delegate;
private volatile boolean started;

GlobalDataStreamSender(Supplier<GetDataStream> delegate, Endpoint endpoint) {
// Ensures that the Supplier is thread-safe
this.delegate = Suppliers.memoize(delegate::get);
GlobalDataStreamSender(GetDataStream delegate, Endpoint endpoint) {
this.delegate = delegate;
this.started = false;
this.endpoint = endpoint;
}

@Override
public GetDataStream get() {
GetDataStream stream() {
if (!started) {
started = true;
// Starting the stream possibly perform IO. Start the stream lazily since not all pipeline
// implementations need to fetch global/side input data.
startStream();
}

return delegate.get();
return delegate;
}

private synchronized void startStream() {
// Check started again after we acquire the lock.
if (!started) {
delegate.start();
started = true;
}
}

@Override
public void close() {
if (started) {
delegate.get().shutdown();
}
delegate.shutdown();
}

Endpoint endpoint() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@
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;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.RpcException;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.WindmillRpcException;
import org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream;
import org.apache.beam.runners.dataflow.worker.windmill.client.commits.WorkCommitter;
import org.apache.beam.runners.dataflow.worker.windmill.client.getdata.GetDataClient;
Expand Down Expand Up @@ -199,7 +199,7 @@ private void applianceDispatchLoop(Supplier<Windmill.GetWorkResponse> getWorkFn)
if (workResponse.getWorkCount() > 0) {
break;
}
} catch (RpcException e) {
} catch (WindmillRpcException e) {
LOG.warn("GetWork failed, retrying:", e);
}
sleepUninterruptibly(backoff, TimeUnit.MILLISECONDS);
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
/*
* 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;

interface StreamSender {
void close();
}
Loading

0 comments on commit 5229c93

Please sign in to comment.