Skip to content

Commit e746fd0

Browse files
committed
add graceful restart mechanism for GetWorkStream to prevent DEADLINE_EXCEEDED; add long deadlines of 1hr to direct streams
1 parent 402c02c commit e746fd0

File tree

5 files changed

+119
-42
lines changed

5 files changed

+119
-42
lines changed

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSender.java

Lines changed: 96 additions & 25 deletions
Original file line numberDiff line numberDiff line change
@@ -20,11 +20,16 @@
2020
import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState;
2121

2222
import java.util.concurrent.CompletableFuture;
23+
import java.util.concurrent.CountDownLatch;
2324
import java.util.concurrent.ExecutorService;
2425
import java.util.concurrent.Executors;
26+
import java.util.concurrent.TimeUnit;
2527
import java.util.concurrent.atomic.AtomicBoolean;
2628
import java.util.concurrent.atomic.AtomicReference;
2729
import java.util.function.Function;
30+
import java.util.function.Supplier;
31+
import javax.annotation.Nullable;
32+
import javax.annotation.concurrent.GuardedBy;
2833
import javax.annotation.concurrent.ThreadSafe;
2934
import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GetWorkRequest;
3035
import org.apache.beam.runners.dataflow.worker.windmill.WindmillConnection;
@@ -41,6 +46,8 @@
4146
import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.FixedStreamHeartbeatSender;
4247
import org.apache.beam.sdk.annotations.Internal;
4348
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
49+
import org.slf4j.Logger;
50+
import org.slf4j.LoggerFactory;
4451

4552
/**
4653
* Owns and maintains a set of streams used to communicate with a specific Windmill worker.
@@ -57,15 +64,26 @@
5764
@Internal
5865
@ThreadSafe
5966
final class WindmillStreamSender implements GetWorkBudgetSpender, StreamSender {
60-
private static final String STREAM_STARTER_THREAD_NAME = "StartWindmillStreamThread-%d";
61-
private final AtomicBoolean started;
62-
private final AtomicReference<GetWorkBudget> getWorkBudget;
63-
private final GetWorkStream getWorkStream;
67+
private static final Logger LOG = LoggerFactory.getLogger(WindmillStreamSender.class);
68+
private static final String STREAM_MANAGER_THREAD_NAME_FORMAT = "WindmillStreamManagerThread";
69+
private static final int GET_WORK_STREAM_TTL_MINUTES = 45;
70+
71+
private final AtomicBoolean isRunning = new AtomicBoolean(false);
6472
private final GetDataStream getDataStream;
6573
private final CommitWorkStream commitWorkStream;
6674
private final WorkCommitter workCommitter;
6775
private final StreamingEngineThrottleTimers streamingEngineThrottleTimers;
6876
private final ExecutorService streamStarter;
77+
private final String backendWorkerToken;
78+
79+
@GuardedBy("activeGetWorkStream")
80+
private final AtomicReference<GetWorkStream> activeGetWorkStream;
81+
82+
@GuardedBy("activeGetWorkStream")
83+
private final AtomicReference<GetWorkBudget> getWorkBudget;
84+
85+
@GuardedBy("activeGetWorkStream")
86+
private final Supplier<GetWorkStream> getWorkStreamFactory;
6987

7088
private WindmillStreamSender(
7189
WindmillConnection connection,
@@ -75,10 +93,9 @@ private WindmillStreamSender(
7593
WorkItemScheduler workItemScheduler,
7694
Function<GetDataStream, GetDataClient> getDataClientFactory,
7795
Function<CommitWorkStream, WorkCommitter> workCommitterFactory) {
78-
this.started = new AtomicBoolean(false);
96+
this.backendWorkerToken = connection.backendWorkerToken();
7997
this.getWorkBudget = getWorkBudget;
8098
this.streamingEngineThrottleTimers = StreamingEngineThrottleTimers.create();
81-
8299
// Stream instances connect/reconnect internally, so we can reuse the same instance through the
83100
// entire lifecycle of WindmillStreamSender.
84101
this.getDataStream =
@@ -88,19 +105,24 @@ private WindmillStreamSender(
88105
streamingEngineStreamFactory.createDirectCommitWorkStream(
89106
connection, streamingEngineThrottleTimers.commitWorkThrottleTimer());
90107
this.workCommitter = workCommitterFactory.apply(commitWorkStream);
91-
this.getWorkStream =
92-
streamingEngineStreamFactory.createDirectGetWorkStream(
93-
connection,
94-
withRequestBudget(getWorkRequest, getWorkBudget.get()),
95-
streamingEngineThrottleTimers.getWorkThrottleTimer(),
96-
FixedStreamHeartbeatSender.create(getDataStream),
97-
getDataClientFactory.apply(getDataStream),
98-
workCommitter,
99-
workItemScheduler);
108+
this.activeGetWorkStream = new AtomicReference<>();
109+
this.getWorkStreamFactory =
110+
() ->
111+
streamingEngineStreamFactory.createDirectGetWorkStream(
112+
connection,
113+
withRequestBudget(getWorkRequest, getWorkBudget.get()),
114+
streamingEngineThrottleTimers.getWorkThrottleTimer(),
115+
FixedStreamHeartbeatSender.create(getDataStream),
116+
getDataClientFactory.apply(getDataStream),
117+
workCommitter,
118+
workItemScheduler);
100119
// 3 threads, 1 for each stream type (GetWork, GetData, CommitWork).
101120
this.streamStarter =
102121
Executors.newFixedThreadPool(
103-
3, new ThreadFactoryBuilder().setNameFormat(STREAM_STARTER_THREAD_NAME).build());
122+
3,
123+
new ThreadFactoryBuilder()
124+
.setNameFormat(STREAM_MANAGER_THREAD_NAME_FORMAT + "-" + backendWorkerToken + "-%d")
125+
.build());
104126
}
105127

106128
static WindmillStreamSender create(
@@ -126,35 +148,49 @@ private static GetWorkRequest withRequestBudget(GetWorkRequest request, GetWorkB
126148
}
127149

128150
synchronized void start() {
129-
if (!started.get()) {
151+
if (isRunning.compareAndSet(false, true)) {
130152
checkState(!streamStarter.isShutdown(), "WindmillStreamSender has already been shutdown.");
131-
132153
// Start these 3 streams in parallel since they each may perform blocking IO.
154+
CountDownLatch waitForInitialStream = new CountDownLatch(1);
155+
streamStarter.execute(() -> getWorkStreamLoop(waitForInitialStream));
133156
CompletableFuture.allOf(
134-
CompletableFuture.runAsync(getWorkStream::start, streamStarter),
135157
CompletableFuture.runAsync(getDataStream::start, streamStarter),
136158
CompletableFuture.runAsync(commitWorkStream::start, streamStarter))
137159
.join();
160+
try {
161+
waitForInitialStream.await();
162+
} catch (InterruptedException e) {
163+
close();
164+
LOG.error("GetWorkStream to {} was never able to start.", backendWorkerToken);
165+
throw new IllegalStateException("GetWorkStream unable to start aborting.", e);
166+
}
138167
workCommitter.start();
139-
started.set(true);
140168
}
141169
}
142170

143171
@Override
144172
public synchronized void close() {
173+
isRunning.set(false);
145174
streamStarter.shutdownNow();
146-
getWorkStream.shutdown();
147175
getDataStream.shutdown();
148176
workCommitter.stop();
149177
commitWorkStream.shutdown();
150178
}
151179

152180
@Override
153181
public void setBudget(long items, long bytes) {
154-
GetWorkBudget budget = GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
155-
getWorkBudget.set(budget);
156-
if (started.get()) {
157-
getWorkStream.setBudget(budget);
182+
synchronized (activeGetWorkStream) {
183+
GetWorkBudget budget = GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
184+
getWorkBudget.set(budget);
185+
if (isRunning.get()) {
186+
@Nullable GetWorkStream stream = activeGetWorkStream.get();
187+
// activeGetWorkStream could be null if start() was called but activeGetWorkStream was not
188+
// populated yet. Populating activeGetWorkStream and setting the budget are guaranteed to
189+
// execute serially since both operations synchronize on activeGetWorkStream.
190+
if (stream != null) {
191+
stream.setBudget(budget);
192+
}
193+
}
158194
}
159195
}
160196

@@ -165,4 +201,39 @@ long getAndResetThrottleTime() {
165201
long getCurrentActiveCommitBytes() {
166202
return workCommitter.currentActiveCommitBytes();
167203
}
204+
205+
/**
206+
* Creates, starts, and gracefully terminates {@link GetWorkStream} before the clientside deadline
207+
* to prevent {@link org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Status#DEADLINE_EXCEEDED} errors.
208+
* If at any point the server closes the stream, reconnects immediately.
209+
*/
210+
private void getWorkStreamLoop(CountDownLatch waitForInitialStream) {
211+
@Nullable GetWorkStream newStream = null;
212+
while (isRunning.get()) {
213+
synchronized (activeGetWorkStream) {
214+
newStream = getWorkStreamFactory.get();
215+
newStream.start();
216+
waitForInitialStream.countDown();
217+
activeGetWorkStream.set(newStream);
218+
}
219+
try {
220+
// Try to gracefully terminate the stream.
221+
if (!newStream.awaitTermination(GET_WORK_STREAM_TTL_MINUTES, TimeUnit.MINUTES)) {
222+
newStream.halfClose();
223+
}
224+
225+
// If graceful termination is unsuccessful, forcefully shutdown.
226+
if (!newStream.awaitTermination(30, TimeUnit.SECONDS)) {
227+
newStream.shutdown();
228+
}
229+
230+
} catch (InterruptedException e) {
231+
// continue until !isRunning.
232+
}
233+
}
234+
235+
if (newStream != null) {
236+
newStream.shutdown();
237+
}
238+
}
168239
}

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/GrpcWindmillStreamFactory.java

Lines changed: 9 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,12 @@ private static <T extends AbstractStub<T>> T withDefaultDeadline(T stub) {
189189
return stub.withDeadlineAfter(DEFAULT_STREAM_RPC_DEADLINE_SECONDS, TimeUnit.SECONDS);
190190
}
191191

192+
private static <T extends AbstractStub<T>> T withLongDeadline(T stub) {
193+
// Deadlines are absolute points in time, so generate a new one everytime this function is
194+
// called.
195+
return stub.withDeadlineAfter(1, TimeUnit.HOURS);
196+
}
197+
192198
private static void printSummaryHtmlForWorker(
193199
String workerToken, Collection<AbstractWindmillStream<?, ?>> streams, PrintWriter writer) {
194200
writer.write(
@@ -230,7 +236,7 @@ public GetWorkStream createDirectGetWorkStream(
230236
WorkItemScheduler workItemScheduler) {
231237
return GrpcDirectGetWorkStream.create(
232238
connection.backendWorkerToken(),
233-
responseObserver -> connection.stub().getWorkStream(responseObserver),
239+
responseObserver -> withLongDeadline(connection.stub()).getWorkStream(responseObserver),
234240
request,
235241
grpcBackOff.get(),
236242
newStreamObserverFactory(),
@@ -265,7 +271,7 @@ public GetDataStream createDirectGetDataStream(
265271
WindmillConnection connection, ThrottleTimer getDataThrottleTimer) {
266272
return GrpcGetDataStream.create(
267273
connection.backendWorkerToken(),
268-
responseObserver -> connection.stub().getDataStream(responseObserver),
274+
responseObserver -> withLongDeadline(connection.stub()).getDataStream(responseObserver),
269275
grpcBackOff.get(),
270276
newStreamObserverFactory(),
271277
streamRegistry,
@@ -297,7 +303,7 @@ public CommitWorkStream createDirectCommitWorkStream(
297303
WindmillConnection connection, ThrottleTimer commitWorkThrottleTimer) {
298304
return GrpcCommitWorkStream.create(
299305
connection.backendWorkerToken(),
300-
responseObserver -> connection.stub().commitWorkStream(responseObserver),
306+
responseObserver -> withLongDeadline(connection.stub()).commitWorkStream(responseObserver),
301307
grpcBackOff.get(),
302308
newStreamObserverFactory(),
303309
streamRegistry,

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/DirectStreamObserver.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ final class DirectStreamObserver<T> implements TerminatingStreamObserver<T> {
4444
private static final long OUTPUT_CHANNEL_CONSIDERED_STALLED_SECONDS = 30;
4545

4646
private final Phaser isReadyNotifier;
47-
private final long deadlineSeconds;
47+
private final long inactivityTimeout;
4848
private final int messagesBetweenIsReadyChecks;
4949
private final Object lock = new Object();
5050

@@ -63,11 +63,11 @@ final class DirectStreamObserver<T> implements TerminatingStreamObserver<T> {
6363
DirectStreamObserver(
6464
Phaser isReadyNotifier,
6565
CallStreamObserver<T> outboundObserver,
66-
long deadlineSeconds,
66+
long inactivityTimeout,
6767
int messagesBetweenIsReadyChecks) {
6868
this.isReadyNotifier = isReadyNotifier;
6969
this.outboundObserver = outboundObserver;
70-
this.deadlineSeconds = deadlineSeconds;
70+
this.inactivityTimeout = inactivityTimeout;
7171
// We always let the first message pass through without blocking because it is performed under
7272
// the StreamPool synchronized block and single header message isn't going to cause memory
7373
// issues due to excessive buffering within grpc.
@@ -153,7 +153,7 @@ public void onNext(T value) {
153153
}
154154
} catch (TimeoutException e) {
155155
totalSecondsWaited += waitSeconds;
156-
if (totalSecondsWaited > deadlineSeconds) {
156+
if (totalSecondsWaited > inactivityTimeout) {
157157
String errorMessage = constructStreamCancelledErrorMessage(totalSecondsWaited);
158158
LOG.error(errorMessage);
159159
throw new WindmillRpcException(errorMessage, e);
@@ -217,11 +217,11 @@ public void terminate(Throwable terminationException) {
217217
}
218218

219219
private String constructStreamCancelledErrorMessage(long totalSecondsWaited) {
220-
return deadlineSeconds > 0
220+
return inactivityTimeout > 0
221221
? "Waited "
222222
+ totalSecondsWaited
223223
+ "s which exceeds given deadline of "
224-
+ deadlineSeconds
224+
+ inactivityTimeout
225225
+ "s for the outboundObserver to become ready meaning "
226226
+ "that the stream deadline was not respected."
227227
: "Output channel has been blocked for "

runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/client/grpc/observers/StreamObserverFactory.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -29,20 +29,20 @@
2929
*/
3030
public abstract class StreamObserverFactory {
3131
public static StreamObserverFactory direct(
32-
long deadlineSeconds, int messagesBetweenIsReadyChecks) {
33-
return new Direct(deadlineSeconds, messagesBetweenIsReadyChecks);
32+
long inactivityTimeout, int messagesBetweenIsReadyChecks) {
33+
return new Direct(inactivityTimeout, messagesBetweenIsReadyChecks);
3434
}
3535

3636
public abstract <ResponseT, RequestT> TerminatingStreamObserver<RequestT> from(
3737
Function<StreamObserver<ResponseT>, StreamObserver<RequestT>> clientFactory,
3838
StreamObserver<ResponseT> responseObserver);
3939

4040
private static class Direct extends StreamObserverFactory {
41-
private final long deadlineSeconds;
41+
private final long inactivityTimeout;
4242
private final int messagesBetweenIsReadyChecks;
4343

44-
Direct(long deadlineSeconds, int messagesBetweenIsReadyChecks) {
45-
this.deadlineSeconds = deadlineSeconds;
44+
Direct(long inactivityTimeout, int messagesBetweenIsReadyChecks) {
45+
this.inactivityTimeout = inactivityTimeout;
4646
this.messagesBetweenIsReadyChecks = messagesBetweenIsReadyChecks;
4747
}
4848

@@ -57,7 +57,7 @@ public <ResponseT, RequestT> TerminatingStreamObserver<RequestT> from(
5757
new ForwardingClientResponseObserver<ResponseT, RequestT>(
5858
inboundObserver, phaser::arrive, phaser::forceTermination));
5959
return new DirectStreamObserver<>(
60-
phaser, outboundObserver, deadlineSeconds, messagesBetweenIsReadyChecks);
60+
phaser, outboundObserver, inactivityTimeout, messagesBetweenIsReadyChecks);
6161
}
6262
}
6363
}

runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/harness/WindmillStreamSenderTest.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
import static org.junit.Assert.assertThrows;
2121
import static org.mockito.ArgumentMatchers.any;
2222
import static org.mockito.ArgumentMatchers.eq;
23+
import static org.mockito.Mockito.atLeastOnce;
2324
import static org.mockito.Mockito.mock;
2425
import static org.mockito.Mockito.spy;
2526
import static org.mockito.Mockito.times;
@@ -226,7 +227,7 @@ public void testCloseAllStreams_closesAllStreams() {
226227
windmillStreamSender.start();
227228
windmillStreamSender.close();
228229

229-
verify(mockGetWorkStream).shutdown();
230+
verify(mockGetWorkStream, atLeastOnce()).shutdown();
230231
verify(mockGetDataStream).shutdown();
231232
verify(mockCommitWorkStream).shutdown();
232233
}
@@ -268,7 +269,6 @@ public void testCloseAllStreams_doesNotStartStreamsAfterClose() {
268269
verify(mockGetDataStream, times(0)).start();
269270
verify(mockCommitWorkStream, times(0)).start();
270271

271-
verify(mockGetWorkStream).shutdown();
272272
verify(mockGetDataStream).shutdown();
273273
verify(mockCommitWorkStream).shutdown();
274274
}

0 commit comments

Comments
 (0)