[
https://issues.apache.org/jira/browse/BEAM-5211?focusedWorklogId=137646&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-137646
]
ASF GitHub Bot logged work on BEAM-5211:
----------------------------------------
Author: ASF GitHub Bot
Created on: 24/Aug/18 02:54
Start Date: 24/Aug/18 02:54
Worklog Time Spent: 10m
Work Description: tweise commented on a change in pull request #6271:
[BEAM-5211] Flink Streaming ExecutableStage operator chain blocks grpc receiver
threads
URL: https://github.com/apache/beam/pull/6271#discussion_r212509932
##########
File path:
runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
##########
@@ -137,12 +137,32 @@ private void
processElementWithSdkHarness(WindowedValue<InputT> element) throws
checkState(
stateRequestHandler != null, "%s not yet prepared",
StateRequestHandler.class.getName());
+ final ArrayList<KV<String, OutputT>> results = new ArrayList();
+
+ OutputReceiverFactory receiverFactory =
+ new OutputReceiverFactory() {
+ @Override
+ public FnDataReceiver<OutputT> create(String pCollectionId) {
+ return (receivedElement) -> {
+ synchronized (results) {
+ // buffer to not block the grpc thread
+ results.add(KV.of(pCollectionId, receivedElement));
+ }
+ };
+ }
+ };
+
try (RemoteBundle bundle =
- stageBundleFactory.getBundle(
- new ReceiverFactory(outputManager, outputMap),
stateRequestHandler, progressHandler)) {
+ stageBundleFactory.getBundle(receiverFactory, stateRequestHandler,
progressHandler)) {
logger.debug(String.format("Sending value: %s", element));
// TODO(BEAM-4681): Add support to Flink to support portable timers.
Iterables.getOnlyElement(bundle.getInputReceivers().values()).accept(element);
+ // TODO: it would be nice to emit results as they arrive, can thread
wait non-blocking?
+ }
+
+ // RemoteBundle close blocks until all results are received
+ for (KV<String, OutputT> result : results) {
Review comment:
Flink operators (and probably their entire chain) cannot be assumed to be
thread-safe. The best solution would be to use the thread that is blocking for
the end of the bundle. The next best thing would be a single output background
thread and synchronize after the bundle is closed. At this time that isn't the
top priority though..
----------------------------------------------------------------
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: 137646)
Time Spent: 1h (was: 50m)
> Flink Streaming ExecutableStage operator chain blocks grpc receiver threads
> ----------------------------------------------------------------------------
>
> Key: BEAM-5211
> URL: https://issues.apache.org/jira/browse/BEAM-5211
> Project: Beam
> Issue Type: Bug
> Components: runner-flink
> Affects Versions: 2.6.0
> Reporter: Thomas Weise
> Assignee: Thomas Weise
> Priority: Major
> Labels: portability
> Fix For: 2.7.0
>
> Attachments: jstack.log
>
> Time Spent: 1h
> Remaining Estimate: 0h
>
> The operator attempts to emit results as they are received, within the grpc
> thread, while the subtask thread is waiting for bundle completion. This leads
> to blocking of grpc threads and eventually deadlock when multiple stages are
> within an operator chain. Observed with wordcount, see attached stacktrace.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)