[
https://issues.apache.org/jira/browse/BEAM-4280?focusedWorklogId=101885&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-101885
]
ASF GitHub Bot logged work on BEAM-4280:
----------------------------------------
Author: ASF GitHub Bot
Created on: 14/May/18 21:12
Start Date: 14/May/18 21:12
Worklog Time Spent: 10m
Work Description: lukecwik closed pull request #5345: [BEAM-4280] Prevent
DirectStreamObserver from blocking indefinitely if invoked from inbound channel
thread
URL: https://github.com/apache/beam/pull/5345
This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:
As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):
diff --git
a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java
b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java
index eb7183f7cea..d098f593e39 100644
---
a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java
+++
b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java
@@ -21,7 +21,11 @@
import io.grpc.stub.CallStreamObserver;
import io.grpc.stub.StreamObserver;
import java.util.concurrent.Phaser;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
import javax.annotation.concurrent.ThreadSafe;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
* A {@link StreamObserver} which uses synchronization on the underlying
@@ -34,21 +38,66 @@
*/
@ThreadSafe
public final class DirectStreamObserver<T> implements StreamObserver<T> {
+ private static final Logger LOGGER =
LoggerFactory.getLogger(DirectStreamObserver.class);
+ private static final int DEFAULT_MAX_MESSAGES_BEFORE_CHECK = 100;
+
private final Phaser phaser;
private final CallStreamObserver<T> outboundObserver;
+ private final int maxMessagesBeforeCheck;
+
+ private int numberOfMessagesBeforeReadyCheck;
public DirectStreamObserver(
Phaser phaser,
CallStreamObserver<T> outboundObserver) {
+ this(phaser, outboundObserver, DEFAULT_MAX_MESSAGES_BEFORE_CHECK);
+ }
+
+ DirectStreamObserver(
+ Phaser phaser,
+ CallStreamObserver<T> outboundObserver,
+ int maxMessagesBeforeCheck) {
this.phaser = phaser;
this.outboundObserver = outboundObserver;
+ this.maxMessagesBeforeCheck = maxMessagesBeforeCheck;
}
@Override
public void onNext(T value) {
- int phase = phaser.getPhase();
- if (!outboundObserver.isReady()) {
- phaser.awaitAdvance(phase);
+ numberOfMessagesBeforeReadyCheck += 1;
+ if (numberOfMessagesBeforeReadyCheck >= maxMessagesBeforeCheck) {
+ numberOfMessagesBeforeReadyCheck = 0;
+ int waitTime = 1;
+ int totalTimeWaited = 0;
+ int phase = phaser.getPhase();
+ while (!outboundObserver.isReady()) {
+ try {
+ phaser.awaitAdvanceInterruptibly(phase, waitTime, TimeUnit.SECONDS);
+ } catch (TimeoutException e) {
+ totalTimeWaited += waitTime;
+ waitTime = waitTime * 2;
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RuntimeException(e);
+ }
+ }
+ if (totalTimeWaited > 0) {
+ // If the phase didn't change, this means that the installed onReady
callback had not
+ // been invoked.
+ if (phase == phaser.getPhase()) {
+ LOGGER.info(
+ "Output channel stalled for {}s, outbound thread {}. See: "
+ + "https://issues.apache.org/jira/browse/BEAM-4280 for the
history for "
+ + "this issue.",
+ totalTimeWaited,
+ Thread.currentThread().getName());
+ } else {
+ LOGGER.debug(
+ "Output channel stalled for {}s, outbound thread {}.",
+ totalTimeWaited,
+ Thread.currentThread().getName());
+ }
+ }
}
synchronized (outboundObserver) {
outboundObserver.onNext(value);
diff --git
a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java
b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java
index bd6c318e7c9..d0a5ed9b97c 100644
---
a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java
+++
b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java
@@ -98,7 +98,8 @@ public void testIsReadyIsHonored() throws Exception {
phaser,
TestStreams.withOnNext((String t) ->
assertTrue(elementsAllowed.get()))
.withIsReady(elementsAllowed::get)
- .build());
+ .build(),
+ 0);
// Start all the tasks
List<Future<String>> results = new ArrayList<>();
@@ -114,7 +115,7 @@ public void testIsReadyIsHonored() throws Exception {
}
// Have them wait and then flip that we do allow elements and wake up
those awaiting
- Uninterruptibles.sleepUninterruptibly(100, TimeUnit.MILLISECONDS);
+ Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
elementsAllowed.set(true);
phaser.arrive();
@@ -123,4 +124,50 @@ public void testIsReadyIsHonored() throws Exception {
}
streamObserver.onCompleted();
}
+
+ /**
+ * This test specifically covers the case if the outbound observer is being
invoked on the same
+ * thread that the inbound observer is. gRPC documentation states:
+ *
+ * <p><i>Note: the onReadyHandler's invocation is serialized on the same
thread pool as the
+ * incoming StreamObserver's onNext(), onError(), and onComplete() handlers.
Blocking the
+ * onReadyHandler will prevent additional messages from being processed by
the incoming
+ * StreamObserver. The onReadyHandler must return in a timely manor or else
message processing
+ * throughput will suffer.
+ * </i>
+ */
+ @Test
+ public void testIsReadyCheckDoesntBlockIfPhaserCallbackNeverHappens() throws
Exception {
+ // Note that we never advance the phaser in this test.
+ final AtomicBoolean elementsAllowed = new AtomicBoolean();
+ final DirectStreamObserver<String> streamObserver =
+ new DirectStreamObserver<>(
+ new AdvancingPhaser(1),
+ TestStreams.withOnNext((String t) ->
assertTrue(elementsAllowed.get()))
+ .withIsReady(elementsAllowed::get)
+ .build(),
+ 0);
+
+ // Start all the tasks
+ List<Future<String>> results = new ArrayList<>();
+ for (final String prefix : ImmutableList.of("0", "1", "2", "3", "4")) {
+ results.add(
+ executor.submit(
+ () -> {
+ for (int i = 0; i < 10; i++) {
+ streamObserver.onNext(prefix + i);
+ }
+ return prefix;
+ }));
+ }
+
+ // Have them wait and then flip that we do allow elements and wake up
those awaiting
+ Uninterruptibles.sleepUninterruptibly(500, TimeUnit.MILLISECONDS);
+ elementsAllowed.set(true);
+
+ for (Future<String> result : results) {
+ result.get();
+ }
+ streamObserver.onCompleted();
+ }
}
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
Issue Time Tracking
-------------------
Worklog Id: (was: 101885)
Time Spent: 1h 20m (was: 1h 10m)
> DirectStreamObserver for outbound channel can block indefinitely if invoked
> from inbound channel thread causing deadlock
> ------------------------------------------------------------------------------------------------------------------------
>
> Key: BEAM-4280
> URL: https://issues.apache.org/jira/browse/BEAM-4280
> Project: Beam
> Issue Type: Bug
> Components: sdk-java-harness
> Reporter: Luke Cwik
> Assignee: Luke Cwik
> Priority: Major
> Labels: portability
> Time Spent: 1h 20m
> Remaining Estimate: 0h
>
> gRPC docs say that:
> // Note: the onReadyHandler's invocation is serialized on the same
> thread pool as the incoming StreamObserver's
> // onNext(), onError(), and onComplete() handlers. Blocking the
> onReadyHandler will prevent additional messages
> // from being processed by the incoming StreamObserver. The
> onReadyHandler must return in a timely manor or else
> // message processing throughput will suffer.
> Looking at the stack, it i because one of the gRPC threads is blocked waiting
> for the channel to become ready, preventing for that same thread to mark it
> as ready:
> "grpc-default-executor-0" #12 daemon prio=5 os_prio=0 tid=0x00007fcea88ee800
> nid=0x3cc8a waiting on condition [0x00007fce4b9f8000]
> java.lang.Thread.State: WAITING (parking)
> at (C/C++) 0x00007fcead7519f2 (Unknown Source)
> at (C/C++) 0x00007fceac8b8f11 (Unknown Source)
> at sun.misc.Unsafe.park(Native Method)
> - parking to wait for <0x0000000740e39c48> (a
> java.util.concurrent.Phaser$QNode)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at java.util.concurrent.Phaser$QNode.block(Phaser.java:1140)
> at
> java.util.concurrent.ForkJoinPool.managedBlock(ForkJoinPool.java:3323)
> at java.util.concurrent.Phaser.internalAwaitAdvance(Phaser.java:1067)
> at java.util.concurrent.Phaser.awaitAdvance(Phaser.java:730)
> at
> org.apache.beam.sdk.fn.stream.DirectStreamObserver.onNext(DirectStreamObserver.java:51)
> at
> org.apache.beam.fn.harness.data.BeamFnDataBufferingOutboundObserver.accept(BeamFnDataBufferingOutboundObserver.java:117)
> at
> org.apache.beam.fn.harness.data.BeamFnDataBufferingOutboundObserver.accept(BeamFnDataBufferingOutboundObserver.java:53)
> at
> org.apache.beam.fn.harness.BeamFnDataWriteRunner.consume(BeamFnDataWriteRunner.java:161)
> at
> org.apache.beam.fn.harness.BeamFnDataWriteRunner$Factory$$Lambda$41/127245540.accept(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.outputTo(FnApiDoFnRunner.java:461)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.access$1200(FnApiDoFnRunner.java:113)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$ProcessBundleContext.output(FnApiDoFnRunner.java:612)
> at
> com.google.cloud.dataflow.integration.synthetic.SyntheticStep.processElement(SyntheticStep.java:93)
> at
> com.google.cloud.dataflow.integration.synthetic.SyntheticStep$DoFnInvoker.invokeProcessElement(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.processElement(FnApiDoFnRunner.java:408)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$$Lambda$51/1126257907.accept(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.outputTo(FnApiDoFnRunner.java:461)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.access$1200(FnApiDoFnRunner.java:113)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$ProcessBundleContext.outputWithTimestamp(FnApiDoFnRunner.java:622)
> at
> org.apache.beam.runners.dataflow.repackaged.org.apache.beam.runners.core.construction.JavaReadViaImpulse$ReadFromBoundedSourceFn.readSoruce(JavaReadViaImpulse.java:139)
> at
> org.apache.beam.runners.dataflow.repackaged.org.apache.beam.runners.core.construction.JavaReadViaImpulse$ReadFromBoundedSourceFn$DoFnInvoker.invokeProcessElement(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.processElement(FnApiDoFnRunner.java:408)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$$Lambda$51/1126257907.accept(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.outputTo(FnApiDoFnRunner.java:461)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.access$1200(FnApiDoFnRunner.java:113)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$ProcessBundleContext.output(FnApiDoFnRunner.java:612)
> at
> org.apache.beam.sdk.transforms.MapElements$1.processElement(MapElements.java:129)
> at
> org.apache.beam.sdk.transforms.MapElements$1$DoFnInvoker.invokeProcessElement(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.processElement(FnApiDoFnRunner.java:408)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$$Lambda$51/1126257907.accept(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.outputTo(FnApiDoFnRunner.java:461)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.access$1200(FnApiDoFnRunner.java:113)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$ProcessBundleContext.output(FnApiDoFnRunner.java:612)
> at
> org.apache.beam.runners.dataflow.ReshuffleOverrideFactory$ReshuffleWithOnlyTrigger$1.processElement(ReshuffleOverrideFactory.java:86)
> at
> org.apache.beam.runners.dataflow.ReshuffleOverrideFactory$ReshuffleWithOnlyTrigger$1$DoFnInvoker.invokeProcessElement(Unknown
> Source)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner.processElement(FnApiDoFnRunner.java:408)
> at
> org.apache.beam.fn.harness.FnApiDoFnRunner$$Lambda$51/1126257907.accept(Unknown
> Source)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.accept(BeamFnDataInboundObserver.java:80)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.accept(BeamFnDataInboundObserver.java:32)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataGrpcMultiplexer$InboundObserver.onNext(BeamFnDataGrpcMultiplexer.java:135)
> at
> org.apache.beam.sdk.fn.data.BeamFnDataGrpcMultiplexer$InboundObserver.onNext(BeamFnDataGrpcMultiplexer.java:123)
> at
> org.apache.beam.sdk.fn.stream.ForwardingClientResponseObserver.onNext(ForwardingClientResponseObserver.java:51)
> at
> io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onMessage(ClientCalls.java:379)
> at
> io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessageRead.runInContext(ClientCallImpl.java:491)
> at io.grpc.internal.ContextRunnable.run(ContextRunnable.java:52)
> at
> io.grpc.internal.SerializingExecutor$TaskRunner.run(SerializingExecutor.java:152)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:748)
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)