gianm commented on code in PR #13373:
URL: https://github.com/apache/druid/pull/13373#discussion_r1024311128


##########
processing/src/main/java/org/apache/druid/frame/channel/ReadableInputStreamFrameChannel.java:
##########
@@ -186,7 +190,15 @@ private void startReading()
                 totalInputStreamBytesRead += bytesRead;
                 if (backpressureFuture != null) {
                   keepReading = false;
-                  backpressureFuture.addListener(() -> keepReading = true, 
Execs.directExecutor());
+                  backpressureFuture.addListener(
+                      () -> {
+                        keepReading = true;
+                        synchronized (readMonitor) {

Review Comment:
   There's a potential race here where:
   
   1) The reader thread sees `!keepReading` on line 153
   2) `keepReading` is set to `true` here
   3) readMonitor is notified here
   4) The reader thread enters the `synchronized (readMonitor)` block on line 
157, and waits for no reason
   
   We can fix it by setting `keepReading` inside this block, and 
double-checking it in the reader thread on entry into the `synchronized 
(readMonitor)` block.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to