[GitHub] [flink] pnowojski commented on a change in pull request #9426: [FLINK-12958] Integrate AsyncWaitOperator with mailbox (preserving compatibility with legacy sources)

2019-08-15 Thread GitBox
pnowojski commented on a change in pull request #9426: [FLINK-12958] Integrate 
AsyncWaitOperator with mailbox (preserving compatibility with legacy sources)
URL: https://github.com/apache/flink/pull/9426#discussion_r314197853
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/Emitter.java
 ##
 @@ -138,19 +145,33 @@ private void output(AsyncResult asyncResult) throws 
InterruptedException {
new Exception("An async 
function call terminated with an exception. " +
"Failing the 
AsyncWaitOperator.", e));
}
+   }
 
-   // remove the peeked element from the async 
collector buffer so that it is no longer
-   // checkpointed
+   // remove the peeked element from the async collector 
buffer so that it is no longer
+   // checkpointed
+   try {
streamElementQueue.poll();
-
-   // notify the main thread that there is again 
space left in the async collector
-   // buffer
-   checkpointLock.notifyAll();
+   } catch (InterruptedException e) {
+   Thread.currentThread().interrupt();
}
+   };
+
+   CompletableFuture writebackFuture = 
CompletableFuture.runAsync(processingRequest, mailboxExecutor);
 
 Review comment:
   Logic of synchronizing on `checkpointLock` and notifying it should also be 
hidden from the user.


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


[GitHub] [flink] pnowojski commented on a change in pull request #9426: [FLINK-12958] Integrate AsyncWaitOperator with mailbox (preserving compatibility with legacy sources)

2019-08-15 Thread GitBox
pnowojski commented on a change in pull request #9426: [FLINK-12958] Integrate 
AsyncWaitOperator with mailbox (preserving compatibility with legacy sources)
URL: https://github.com/apache/flink/pull/9426#discussion_r314195257
 
 

 ##
 File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/async/AsyncWaitOperator.java
 ##
 @@ -397,26 +404,41 @@ private void stopResources(boolean waitForShutdown) 
throws InterruptedException
 * @throws InterruptedException if the current thread has been 
interrupted
 */
private  void addAsyncBufferEntry(StreamElementQueueEntry 
streamElementQueueEntry) throws InterruptedException {
-   assert(Thread.holdsLock(checkpointingLock));
-
pendingStreamElementQueueEntry = streamElementQueueEntry;
 
-   while (!queue.tryPut(streamElementQueueEntry)) {
-   // we wait for the emitter to notify us if the queue 
has space left again
-   checkpointingLock.wait();
+   if (!queue.tryPut(streamElementQueueEntry)) {
+   do {
+   // We will receive a notify on the lock at the 
end of each processing-letter execution (see {Emitter})
+   // so we can try to put again.
+   if (!tryYieldMainThreadExecutor()) {
 
 Review comment:
   This logic (difference of mailbox vs non mailbox thread) should be 
encapsulated in the mailbox. It's already duplicated here, it will be 
duplicated in other places (like `ContinousFileReaderOperator`) and mailbox 
should leak the details of the threading model to the operators.
   
   something like `mailbox.yield()` called from here, should do this black 
magic in a secret.


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


[GitHub] [flink] pnowojski commented on a change in pull request #9426: [FLINK-12958] Integrate AsyncWaitOperator with mailbox (preserving compatibility with legacy sources)

2019-08-15 Thread GitBox
pnowojski commented on a change in pull request #9426: [FLINK-12958] Integrate 
AsyncWaitOperator with mailbox (preserving compatibility with legacy sources)
URL: https://github.com/apache/flink/pull/9426#discussion_r314198520
 
 

 ##
 File path: 
flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
 ##
 @@ -111,8 +111,8 @@
protected StreamTestSingleInputGate[] inputGates;
 
public StreamTaskTestHarness(
-   Function> 
taskFactory,
-   TypeInformation outputType) {
+   Function> taskFactory,
 
 Review comment:
   Could you revert the whitespace changes? They are questionable for me and 
even if we want them, they shouldn't be part of the same commit.


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