Skip to content

Commit e7203a9

Browse files
committed
address PR comments
1 parent e746fd0 commit e7203a9

File tree

3 files changed

+94
-61
lines changed

3 files changed

+94
-61
lines changed

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

Lines changed: 78 additions & 51 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@
2525
import java.util.concurrent.Executors;
2626
import java.util.concurrent.TimeUnit;
2727
import java.util.concurrent.atomic.AtomicBoolean;
28-
import java.util.concurrent.atomic.AtomicReference;
2928
import java.util.function.Function;
3029
import java.util.function.Supplier;
3130
import javax.annotation.Nullable;
@@ -44,6 +43,7 @@
4443
import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudget;
4544
import org.apache.beam.runners.dataflow.worker.windmill.work.budget.GetWorkBudgetSpender;
4645
import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.FixedStreamHeartbeatSender;
46+
import org.apache.beam.runners.dataflow.worker.windmill.work.refresh.HeartbeatSender;
4747
import org.apache.beam.sdk.annotations.Internal;
4848
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
4949
import org.slf4j.Logger;
@@ -66,35 +66,40 @@
6666
final class WindmillStreamSender implements GetWorkBudgetSpender, StreamSender {
6767
private static final Logger LOG = LoggerFactory.getLogger(WindmillStreamSender.class);
6868
private static final String STREAM_MANAGER_THREAD_NAME_FORMAT = "WindmillStreamManagerThread";
69+
70+
// Must be shorter than withLongDeadline() duration in GrpcWindmillStreamFactory to prevent
71+
// DEADLINE_EXCEEDED.
6972
private static final int GET_WORK_STREAM_TTL_MINUTES = 45;
7073

7174
private final AtomicBoolean isRunning = new AtomicBoolean(false);
7275
private final GetDataStream getDataStream;
7376
private final CommitWorkStream commitWorkStream;
7477
private final WorkCommitter workCommitter;
7578
private final StreamingEngineThrottleTimers streamingEngineThrottleTimers;
76-
private final ExecutorService streamStarter;
79+
private final ExecutorService streamManagerExecutor;
7780
private final String backendWorkerToken;
7881

79-
@GuardedBy("activeGetWorkStream")
80-
private final AtomicReference<GetWorkStream> activeGetWorkStream;
81-
82-
@GuardedBy("activeGetWorkStream")
83-
private final AtomicReference<GetWorkBudget> getWorkBudget;
82+
private final Object streamLock = new Object();
8483

85-
@GuardedBy("activeGetWorkStream")
84+
@GuardedBy("streamLock")
8685
private final Supplier<GetWorkStream> getWorkStreamFactory;
8786

87+
@GuardedBy("streamLock")
88+
private @Nullable GetWorkStream activeGetWorkStream;
89+
90+
@GuardedBy("streamLock")
91+
private GetWorkBudget getWorkBudget;
92+
8893
private WindmillStreamSender(
8994
WindmillConnection connection,
9095
GetWorkRequest getWorkRequest,
91-
AtomicReference<GetWorkBudget> getWorkBudget,
96+
GetWorkBudget initialGetWorkBudget,
9297
GrpcWindmillStreamFactory streamingEngineStreamFactory,
9398
WorkItemScheduler workItemScheduler,
9499
Function<GetDataStream, GetDataClient> getDataClientFactory,
95100
Function<CommitWorkStream, WorkCommitter> workCommitterFactory) {
96101
this.backendWorkerToken = connection.backendWorkerToken();
97-
this.getWorkBudget = getWorkBudget;
102+
this.getWorkBudget = initialGetWorkBudget;
98103
this.streamingEngineThrottleTimers = StreamingEngineThrottleTimers.create();
99104
// Stream instances connect/reconnect internally, so we can reuse the same instance through the
100105
// entire lifecycle of WindmillStreamSender.
@@ -105,21 +110,26 @@ private WindmillStreamSender(
105110
streamingEngineStreamFactory.createDirectCommitWorkStream(
106111
connection, streamingEngineThrottleTimers.commitWorkThrottleTimer());
107112
this.workCommitter = workCommitterFactory.apply(commitWorkStream);
108-
this.activeGetWorkStream = new AtomicReference<>();
113+
this.activeGetWorkStream = null;
114+
115+
HeartbeatSender heartbeatSender = FixedStreamHeartbeatSender.create(getDataStream);
116+
GetDataClient getDataClient = getDataClientFactory.apply(getDataStream);
109117
this.getWorkStreamFactory =
110-
() ->
111-
streamingEngineStreamFactory.createDirectGetWorkStream(
118+
() -> {
119+
synchronized (streamLock) {
120+
return streamingEngineStreamFactory.createDirectGetWorkStream(
112121
connection,
113-
withRequestBudget(getWorkRequest, getWorkBudget.get()),
122+
withRequestBudget(getWorkRequest, getWorkBudget),
114123
streamingEngineThrottleTimers.getWorkThrottleTimer(),
115-
FixedStreamHeartbeatSender.create(getDataStream),
116-
getDataClientFactory.apply(getDataStream),
124+
heartbeatSender,
125+
getDataClient,
117126
workCommitter,
118127
workItemScheduler);
128+
}
129+
};
119130
// 3 threads, 1 for each stream type (GetWork, GetData, CommitWork).
120-
this.streamStarter =
121-
Executors.newFixedThreadPool(
122-
3,
131+
this.streamManagerExecutor =
132+
Executors.newCachedThreadPool(
123133
new ThreadFactoryBuilder()
124134
.setNameFormat(STREAM_MANAGER_THREAD_NAME_FORMAT + "-" + backendWorkerToken + "-%d")
125135
.build());
@@ -128,15 +138,15 @@ private WindmillStreamSender(
128138
static WindmillStreamSender create(
129139
WindmillConnection connection,
130140
GetWorkRequest getWorkRequest,
131-
GetWorkBudget getWorkBudget,
141+
GetWorkBudget initialGetWorkBudget,
132142
GrpcWindmillStreamFactory streamingEngineStreamFactory,
133143
WorkItemScheduler workItemScheduler,
134144
Function<GetDataStream, GetDataClient> getDataClientFactory,
135145
Function<CommitWorkStream, WorkCommitter> workCommitterFactory) {
136146
return new WindmillStreamSender(
137147
connection,
138148
getWorkRequest,
139-
new AtomicReference<>(getWorkBudget),
149+
initialGetWorkBudget,
140150
streamingEngineStreamFactory,
141151
workItemScheduler,
142152
getDataClientFactory,
@@ -149,13 +159,14 @@ private static GetWorkRequest withRequestBudget(GetWorkRequest request, GetWorkB
149159

150160
synchronized void start() {
151161
if (isRunning.compareAndSet(false, true)) {
152-
checkState(!streamStarter.isShutdown(), "WindmillStreamSender has already been shutdown.");
162+
checkState(
163+
!streamManagerExecutor.isShutdown(), "WindmillStreamSender has already been shutdown.");
153164
// Start these 3 streams in parallel since they each may perform blocking IO.
154165
CountDownLatch waitForInitialStream = new CountDownLatch(1);
155-
streamStarter.execute(() -> getWorkStreamLoop(waitForInitialStream));
166+
streamManagerExecutor.execute(() -> getWorkStreamLoop(waitForInitialStream::countDown));
156167
CompletableFuture.allOf(
157-
CompletableFuture.runAsync(getDataStream::start, streamStarter),
158-
CompletableFuture.runAsync(commitWorkStream::start, streamStarter))
168+
CompletableFuture.runAsync(getDataStream::start, streamManagerExecutor),
169+
CompletableFuture.runAsync(commitWorkStream::start, streamManagerExecutor))
159170
.join();
160171
try {
161172
waitForInitialStream.await();
@@ -171,24 +182,23 @@ synchronized void start() {
171182
@Override
172183
public synchronized void close() {
173184
isRunning.set(false);
174-
streamStarter.shutdownNow();
185+
streamManagerExecutor.shutdownNow();
175186
getDataStream.shutdown();
176187
workCommitter.stop();
177188
commitWorkStream.shutdown();
178189
}
179190

180191
@Override
181192
public void setBudget(long items, long bytes) {
182-
synchronized (activeGetWorkStream) {
183-
GetWorkBudget budget = GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
184-
getWorkBudget.set(budget);
193+
GetWorkBudget budget = GetWorkBudget.builder().setItems(items).setBytes(bytes).build();
194+
synchronized (streamLock) {
195+
getWorkBudget = budget;
185196
if (isRunning.get()) {
186-
@Nullable GetWorkStream stream = activeGetWorkStream.get();
187197
// activeGetWorkStream could be null if start() was called but activeGetWorkStream was not
188198
// populated yet. Populating activeGetWorkStream and setting the budget are guaranteed to
189199
// execute serially since both operations synchronize on activeGetWorkStream.
190-
if (stream != null) {
191-
stream.setBudget(budget);
200+
if (activeGetWorkStream != null) {
201+
activeGetWorkStream.setBudget(budget);
192202
}
193203
}
194204
}
@@ -207,33 +217,50 @@ long getCurrentActiveCommitBytes() {
207217
* to prevent {@link org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Status#DEADLINE_EXCEEDED} errors.
208218
* If at any point the server closes the stream, reconnects immediately.
209219
*/
210-
private void getWorkStreamLoop(CountDownLatch waitForInitialStream) {
211-
@Nullable GetWorkStream newStream = null;
220+
private void getWorkStreamLoop(Runnable onInitialStream) {
212221
while (isRunning.get()) {
213-
synchronized (activeGetWorkStream) {
214-
newStream = getWorkStreamFactory.get();
222+
CountDownLatch triggerNewStream = new CountDownLatch(1);
223+
synchronized (streamLock) {
224+
GetWorkStream newStream = getWorkStreamFactory.get();
215225
newStream.start();
216-
waitForInitialStream.countDown();
217-
activeGetWorkStream.set(newStream);
226+
onInitialStream.run();
227+
activeGetWorkStream = newStream;
228+
// Offload the old stream termination to a different thread which will terminate once it
229+
// closes the stream. This allows this thread to not wait for the old stream to terminate
230+
// before creating a new stream once the old stream has timed out.
231+
streamManagerExecutor.execute(() -> gracefullyTerminateStream(newStream, triggerNewStream));
218232
}
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-
}
229233

234+
// Wait for a new stream to be triggered w/o holding the lock.
235+
try {
236+
triggerNewStream.await();
230237
} catch (InterruptedException e) {
231-
// continue until !isRunning.
238+
assert !isRunning.get();
232239
}
233240
}
241+
}
242+
243+
private void gracefullyTerminateStream(GetWorkStream stream, CountDownLatch triggerNewStream) {
244+
try {
245+
// Try to gracefully terminate the stream.
246+
if (!stream.awaitTermination(GET_WORK_STREAM_TTL_MINUTES, TimeUnit.MINUTES)) {
247+
// Free any threads waiting to create a new stream.
248+
triggerNewStream.countDown();
249+
stream.halfClose();
250+
}
251+
252+
// Wait a bit for retries/drains then forcefully shutdown if graceful termination is
253+
// unsuccessful.
254+
if (!stream.awaitTermination(5, TimeUnit.MINUTES)) {
255+
stream.shutdown();
256+
}
234257

235-
if (newStream != null) {
236-
newStream.shutdown();
258+
} catch (InterruptedException e) {
259+
assert !isRunning.get();
260+
} finally {
261+
// Make sure we clean up the stream.
262+
stream.shutdown();
263+
triggerNewStream.countDown();
237264
}
238265
}
239266
}

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

Lines changed: 13 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -69,7 +69,7 @@
6969
*/
7070
@ThreadSafe
7171
@Internal
72-
public class GrpcWindmillStreamFactory implements StatusDataProvider {
72+
public final class GrpcWindmillStreamFactory implements StatusDataProvider {
7373

7474
private static final long DEFAULT_STREAM_RPC_DEADLINE_SECONDS = 300;
7575
private static final Duration MIN_BACKOFF = Duration.millis(1);
@@ -189,7 +189,11 @@ 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) {
192+
/**
193+
* Set a longer deadline for directpath since we have explicit semantics on when to open and close
194+
* the streams w/ the fan out metadata.
195+
*/
196+
private static <T extends AbstractStub<T>> T withDirectPathDeadline(T stub) {
193197
// Deadlines are absolute points in time, so generate a new one everytime this function is
194198
// called.
195199
return stub.withDeadlineAfter(1, TimeUnit.HOURS);
@@ -236,7 +240,8 @@ public GetWorkStream createDirectGetWorkStream(
236240
WorkItemScheduler workItemScheduler) {
237241
return GrpcDirectGetWorkStream.create(
238242
connection.backendWorkerToken(),
239-
responseObserver -> withLongDeadline(connection.stub()).getWorkStream(responseObserver),
243+
responseObserver ->
244+
withDirectPathDeadline(connection.stub()).getWorkStream(responseObserver),
240245
request,
241246
grpcBackOff.get(),
242247
newStreamObserverFactory(),
@@ -271,7 +276,8 @@ public GetDataStream createDirectGetDataStream(
271276
WindmillConnection connection, ThrottleTimer getDataThrottleTimer) {
272277
return GrpcGetDataStream.create(
273278
connection.backendWorkerToken(),
274-
responseObserver -> withLongDeadline(connection.stub()).getDataStream(responseObserver),
279+
responseObserver ->
280+
withDirectPathDeadline(connection.stub()).getDataStream(responseObserver),
275281
grpcBackOff.get(),
276282
newStreamObserverFactory(),
277283
streamRegistry,
@@ -303,7 +309,8 @@ public CommitWorkStream createDirectCommitWorkStream(
303309
WindmillConnection connection, ThrottleTimer commitWorkThrottleTimer) {
304310
return GrpcCommitWorkStream.create(
305311
connection.backendWorkerToken(),
306-
responseObserver -> withLongDeadline(connection.stub()).commitWorkStream(responseObserver),
312+
responseObserver ->
313+
withDirectPathDeadline(connection.stub()).commitWorkStream(responseObserver),
307314
grpcBackOff.get(),
308315
newStreamObserverFactory(),
309316
streamRegistry,
@@ -346,7 +353,7 @@ public void appendSummaryHtml(PrintWriter writer) {
346353
}
347354

348355
@VisibleForTesting
349-
final ImmutableSet<AbstractWindmillStream<?, ?>> streamRegistry() {
356+
ImmutableSet<AbstractWindmillStream<?, ?>> streamRegistry() {
350357
return ImmutableSet.copyOf(streamRegistry);
351358
}
352359

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

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -161,7 +161,7 @@ public void onNext(T value) {
161161

162162
if (totalSecondsWaited > OUTPUT_CHANNEL_CONSIDERED_STALLED_SECONDS) {
163163
LOG.info(
164-
"Output channel stalled for {}s, outbound thread {}.",
164+
"Output channel stalled for {}s waiting to be ready, outbound thread {}.",
165165
totalSecondsWaited,
166166
Thread.currentThread().getName());
167167
}
@@ -220,10 +220,9 @@ private String constructStreamCancelledErrorMessage(long totalSecondsWaited) {
220220
return inactivityTimeout > 0
221221
? "Waited "
222222
+ totalSecondsWaited
223-
+ "s which exceeds given deadline of "
223+
+ "s which exceeds given timeout of "
224224
+ inactivityTimeout
225-
+ "s for the outboundObserver to become ready meaning "
226-
+ "that the stream deadline was not respected."
225+
+ "s for the outboundObserver to become ready."
227226
: "Output channel has been blocked for "
228227
+ totalSecondsWaited
229228
+ "s. Restarting stream internally.";

0 commit comments

Comments
 (0)