dawidwys commented on a change in pull request #16589:
URL: https://github.com/apache/flink/pull/16589#discussion_r680306640



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
##########
@@ -281,15 +320,50 @@ protected void declineCheckpoint(long checkpointId) {
         public void run() {
             try {
                 mainOperator.run(lock, operatorChain);
-                if (!wasStoppedExternally && !isCanceled()) {
-                    synchronized (lock) {
-                        operatorChain.setIgnoreEndOfInput(false);
-                    }
-                }
+                completeProcessing();
                 completionFuture.complete(null);
             } catch (Throwable t) {
                 // Note, t can be also an InterruptedException
-                completionFuture.completeExceptionally(t);
+                if (isCanceled()
+                        && ExceptionUtils.findThrowable(t, 
InterruptedException.class)
+                                .isPresent()) {
+                    completionFuture.completeExceptionally(new 
CancelTaskException(t));
+                } else if (finishingReason == 
FinishingReason.STOP_WITH_SAVEPOINT_DRAIN
+                        && ExceptionUtils.findThrowable(t, 
InterruptedException.class)
+                                .isPresent()) {
+                    // if we are stopping the source thread for 
stop-with-savepoint
+                    // we may actually return from run with an 
InterruptedException which
+                    // should be ignored (e.g. Kinesis case see FLINK-23528)
+                    try {
+                        // clear the interrupted status for the thread
+                        Thread.interrupted();
+                        completeProcessing();
+                        completionFuture.complete(null);
+                    } catch (Throwable e) {
+                        completionFuture.completeExceptionally(e);
+                    }

Review comment:
       The next line in the current code:
   ```
    else if (finishingReason == FinishingReason.STOP_WITH_SAVEPOINT_NO_DRAIN) {
                       // swallow all exceptions if the source was stopped 
without drain
                       completionFuture.complete(null);
                   }
   ```
   or on master 
(https://github.com/apache/flink/blob/3b115544b04572831e162288097105c63ca5e048/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java#L183):
   ```
           sourceThread
                   .getCompletionFuture()
                   .whenComplete(
                           (Void ignore, Throwable sourceThreadThrowable) -> {
                               if (isCanceled()
                                       && ExceptionUtils.findThrowable(
                                                       sourceThreadThrowable,
                                                       
InterruptedException.class)
                                               .isPresent()) {
                                   mailboxProcessor.reportThrowable(
                                           new 
CancelTaskException(sourceThreadThrowable));
                               } else if (**!wasStoppedExternally** && 
sourceThreadThrowable != null) {
                                   
mailboxProcessor.reportThrowable(sourceThreadThrowable);
                               } else {
                                   mailboxProcessor.suspend();
                               }
                           });
   ```




-- 
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]


Reply via email to