Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

add shutdown and start mechanics to windmill streams #32774

Open
wants to merge 22 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 13 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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,6 @@
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 java.util.HashSet;
import java.util.Map.Entry;
import java.util.NoSuchElementException;
Expand Down Expand Up @@ -112,7 +111,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 +142,6 @@ private FanOutStreamingEngineWorkerHarness(
this.totalGetWorkBudget = totalGetWorkBudget;
this.activeMetadataVersion = Long.MIN_VALUE;
this.workCommitterFactory = workCommitterFactory;
this.getWorkerMetadataStream = null;
}

/**
Expand Down Expand Up @@ -204,9 +202,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 +224,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 Down Expand Up @@ -320,7 +319,7 @@ private void closeStreamsNotIn(WindmillEndpoints newWindmillEndpoints) {
windmillStreamManager.execute(() -> closeStreamSender(sender.endpoint(), sender)));
}

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 +345,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 +393,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 +415,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();
scwhittle marked this conversation as resolved.
Show resolved Hide resolved
}

return delegate.get();
return delegate;
}

private synchronized void startStream() {
// Check started again after we acquire the lock.
if (!started) {
delegate.start();
scwhittle marked this conversation as resolved.
Show resolved Hide resolved
started = true;
}
}

@Override
public void close() {
if (started) {
delegate.get().shutdown();
}
delegate.shutdown();
scwhittle marked this conversation as resolved.
Show resolved Hide resolved
}

Endpoint endpoint() {
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();
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,14 @@
*/
package org.apache.beam.runners.dataflow.worker.streaming.harness;

import java.io.Closeable;
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;

import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Function;
import java.util.function.Supplier;
import javax.annotation.concurrent.ThreadSafe;
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest;
import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection;
Expand All @@ -37,20 +40,13 @@
import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetSpender;
import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.FixedStreamHeartbeatSender;
import org.apache.beam.sdk.annotations.Internal;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Suppliers;
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;

/**
* Owns and maintains a set of streams used to communicate with a specific Windmill worker.
* Underlying streams are "cached" in a threadsafe manner so that once {@link Supplier#get} is
* called, a stream that is already started is returned.
*
* <p>Holds references to {@link
* Supplier<org.apache.beam.runners.dataflow.worker.windmill.client.WindmillStream>} because
* initializing the streams automatically start them, and we want to do so lazily here once the
* {@link GetWorkBudget} is set.
*
* <p>Once started, the underlying streams are "alive" until they are manually closed via {@link
* #close()} ()}.
* #close()}.
*
* <p>If closed, it means that the backend endpoint is no longer in the worker set. Once closed,
* these instances are not reused.
Expand All @@ -60,14 +56,16 @@
*/
@Internal
@ThreadSafe
final class WindmillStreamSender implements GetWorkBudgetSpender, Closeable {
final class WindmillStreamSender implements GetWorkBudgetSpender, StreamSender {
private static final String STREAM_STARTER_THREAD_NAME = "StartWindmillStreamThread-%d";
private final AtomicBoolean started;
private final AtomicReference<GetWorkBudget> getWorkBudget;
private final Supplier<GetWorkStream> getWorkStream;
private final Supplier<GetDataStream> getDataStream;
private final Supplier<CommitWorkStream> commitWorkStream;
private final Supplier<WorkCommitter> workCommitter;
private final GetWorkStream getWorkStream;
private final GetDataStream getDataStream;
private final CommitWorkStream commitWorkStream;
private final WorkCommitter workCommitter;
private final StreamingEngineThrottleTimers streamingEngineThrottleTimers;
private final ExecutorService streamStarter;

private WindmillStreamSender(
WindmillConnection connection,
Expand All @@ -81,33 +79,28 @@ private WindmillStreamSender(
this.getWorkBudget = getWorkBudget;
this.streamingEngineThrottleTimers = StreamingEngineThrottleTimers.create();

// All streams are memoized/cached since they are expensive to create and some implementations
// perform side effects on construction (i.e. sending initial requests to the stream server to
// initiate the streaming RPC connection). Stream instances connect/reconnect internally, so we
// can reuse the same instance through the entire lifecycle of WindmillStreamSender.
// Stream instances connect/reconnect internally, so we can reuse the same instance through the
// entire lifecycle of WindmillStreamSender.
this.getDataStream =
Suppliers.memoize(
() ->
streamingEngineStreamFactory.createGetDataStream(
connection.stub(), streamingEngineThrottleTimers.getDataThrottleTimer()));
streamingEngineStreamFactory.createDirectGetDataStream(
connection, streamingEngineThrottleTimers.getDataThrottleTimer());
this.commitWorkStream =
Suppliers.memoize(
() ->
streamingEngineStreamFactory.createCommitWorkStream(
connection.stub(), streamingEngineThrottleTimers.commitWorkThrottleTimer()));
this.workCommitter =
Suppliers.memoize(() -> workCommitterFactory.apply(commitWorkStream.get()));
streamingEngineStreamFactory.createDirectCommitWorkStream(
connection, streamingEngineThrottleTimers.commitWorkThrottleTimer());
this.workCommitter = workCommitterFactory.apply(commitWorkStream);
this.getWorkStream =
Suppliers.memoize(
() ->
streamingEngineStreamFactory.createDirectGetWorkStream(
connection,
withRequestBudget(getWorkRequest, getWorkBudget.get()),
streamingEngineThrottleTimers.getWorkThrottleTimer(),
FixedStreamHeartbeatSender.create(getDataStream.get()),
getDataClientFactory.apply(getDataStream.get()),
workCommitter.get(),
workItemScheduler));
streamingEngineStreamFactory.createDirectGetWorkStream(
connection,
withRequestBudget(getWorkRequest, getWorkBudget.get()),
streamingEngineThrottleTimers.getWorkThrottleTimer(),
FixedStreamHeartbeatSender.create(getDataStream),
getDataClientFactory.apply(getDataStream),
workCommitter,
workItemScheduler);
// 3 threads, 1 for each stream type (GetWork, GetData, CommitWork).
this.streamStarter =
Executors.newFixedThreadPool(
3, new ThreadFactoryBuilder().setNameFormat(STREAM_STARTER_THREAD_NAME).build());
}

static WindmillStreamSender create(
Expand All @@ -132,34 +125,35 @@ private static GetWorkRequest withRequestBudget(GetWorkRequest request, GetWorkB
return request.toBuilder().setMaxItems(budget.items()).setMaxBytes(budget.bytes()).build();
}

@SuppressWarnings("ReturnValueIgnored")
void startStreams() {
getWorkStream.get();
getDataStream.get();
commitWorkStream.get();
workCommitter.get().start();
// *stream.get() is all memoized in a threadsafe manner.
started.set(true);
synchronized void start() {
if (!started.get()) {
checkState(!streamStarter.isShutdown(), "WindmillStreamSender has already been shutdown.");
// Start these 3 streams in parallel since they each may perform blocking IO.
CompletableFuture.allOf(
CompletableFuture.runAsync(getWorkStream::start, streamStarter),
CompletableFuture.runAsync(getDataStream::start, streamStarter),
CompletableFuture.runAsync(commitWorkStream::start, streamStarter))
.join();
workCommitter.start();
started.set(true);
}
}

@Override
public void close() {
// Supplier<Stream>.get() starts the stream which is an expensive operation as it initiates the
// streaming RPCs by possibly making calls over the network. Do not close the streams unless
// they have already been started.
if (started.get()) {
getWorkStream.get().shutdown();
getDataStream.get().shutdown();
workCommitter.get().stop();
commitWorkStream.get().shutdown();
}
public synchronized void close() {
streamStarter.shutdownNow();
getWorkStream.shutdown();
getDataStream.shutdown();
workCommitter.stop();
commitWorkStream.shutdown();
}

@Override
public void setBudget(long items, long bytes) {
getWorkBudget.set(getWorkBudget.get().apply(items, bytes));
GetWorkBudget budget = GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
getWorkBudget.set(budget);
if (started.get()) {
getWorkStream.get().setBudget(items, bytes);
getWorkStream.setBudget(budget);
}
}

Expand All @@ -168,6 +162,6 @@ long getAndResetThrottleTime() {
}

long getCurrentActiveCommitBytes() {
return started.get() ? workCommitter.get().currentActiveCommitBytes() : 0;
return workCommitter.currentActiveCommitBytes();
}
}
Loading
Loading