pnowojski commented on a change in pull request #9564: 
[FLINK-12481][FLINK-12482][FLINK-12958] Streaming runtime: integrate mailbox 
for timer triggers, checkpoints and AsyncWaitOperator
URL: https://github.com/apache/flink/pull/9564#discussion_r319833048
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
 ##########
 @@ -1139,17 +1138,9 @@ public void triggerCheckpointBarrier(
 
                if (executionState == ExecutionState.RUNNING && invokable != 
null) {
 
-                       // build a local closure
-                       final SafetyNetCloseableRegistry 
safetyNetCloseableRegistry =
-                               
FileSystemSafetyNet.getSafetyNetCloseableRegistryForThread();
-
                        Runnable runnable = new Runnable() {
                                @Override
                                public void run() {
-                                       // set safety net from the task's 
context for checkpointing thread
 
 Review comment:
   > Do you have any reasons to keep it?
   
   I don't know this code and I'm not sure how it's suppose to work, but I see 
that it's being used 
(`FileSystemSafetyNet#initializeSafetyNetForThread/closeSafetyNetAndGuardedResourcesForThread/wrapWithSafetyNetWhenActivated`).
 Since you were removing it, I thought that you have investigated how is it 
suppose to be working and that's safe to remove it (as I'm pretty sure that 
such safety feature has a very poor test coverage).
   
   > For the second part of your comment, I don't think it would be correct to 
allow any other (non main task thread) to get letters from the mailbox. And so 
far, we haven't raised any reason to have such requirement (that letters can be 
run by some other threads).
   
   I guess that's right as long as we do not allow for yielding in the legacy 
source threads. Do we have a `checkstate` somewhere for that? Somewhere Inside 
yield a `checkState(isMailboxThread())`?

----------------------------------------------------------------
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:
[email protected]


With regards,
Apache Git Services

Reply via email to