pnowojski commented on code in PR #20151:
URL: https://github.com/apache/flink/pull/20151#discussion_r1052418421
##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequestExecutorImpl.java:
##########
@@ -51,27 +66,40 @@ class ChannelStateWriteRequestExecutorImpl implements
ChannelStateWriteRequestEx
private final Thread thread;
private volatile Exception thrown = null;
private volatile boolean wasClosed = false;
- private final String taskName;
+
+ private final Map<SubtaskID, Queue<ChannelStateWriteRequest>>
unreadyQueues =
+ new ConcurrentHashMap<>();
+
+ private final JobID jobID;
+ private final Set<SubtaskID> subtasks;
+ private final AtomicBoolean isRegistering = new AtomicBoolean(true);
+ private final int numberOfSubtasksShareFile;
Review Comment:
> BlockingDeque.take() will wait until an element becomes available. Deque
is hard to achieve. BlockingDeque is easy to implement the producer & consumer
model.
That's a slight complexity, but should be easily solved via `lock.wait()`
and `lock.notifyAll()` called in one or two places (`close()` and whenever we
add anything to the current `dequeue`)?
https://howtodoinjava.com/java/multi-threading/wait-notify-and-notifyall-methods/
The loop probably should look like this:
```
while (true) {
synchronized (lock) {
if (wasClosed) return;
(...)
ChannelStateWriteRequest request = waitAndTakeUnsafe();
(...)
}
}
private ChannelStateWriteRequest waitAndTakeUnsafe() {
ChannelStateWriteRequest request;
while (true) {
request = dequeue.pollFirst();
if (request == null) {
lock.wait();
}
else {
return request;
}
}
```
--
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]