rkhachatryan commented on a change in pull request #11507: [FLINK-16587] Add 
basic CheckpointBarrierHandler for unaligned checkpoint
URL: https://github.com/apache/flink/pull/11507#discussion_r407979727
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/RemoteInputChannel.java
 ##########
 @@ -513,8 +540,15 @@ public void onBuffer(Buffer buffer, int sequenceNumber, 
int backlog) throws IOEx
                boolean recycleBuffer = true;
 
                try {
+                       if (expectedSequenceNumber != sequenceNumber) {
+                               onError(new 
BufferReorderingException(expectedSequenceNumber, sequenceNumber));
+                               return;
+                       }
 
                        final boolean wasEmpty;
+                       CheckpointBarrier notifyReceivedBarrier = null;
+                       Buffer notifyReceivedBuffer = null;
+                       final boolean hasListener = 
inputGate.bufferReceivedListener != null;
 
 Review comment:
   I'm concerned about thread-safety of `inputGate.bufferReceivedListener`.
   It's not `final` or `volatile` and can be updated by the other (task) thread.
   (this is also valid for the actual use of it below when notifying)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to