reswqa commented on code in PR #23179:
URL: https://github.com/apache/flink/pull/23179#discussion_r1295350734
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImpl.java:
##########
@@ -329,13 +327,15 @@ private Optional<BufferIndexOrError>
checkAndGetFirstBufferIndexOrError(
return Optional.ofNullable(peek);
}
- private void increaseBacklog(Buffer buffer) {
+ @GuardedBy("lock")
Review Comment:
Should be removed as `backlog` is not guarded by `lock` anymore.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionConsumerMemoryDataManager.java:
##########
@@ -44,7 +44,6 @@ public class HsSubpartitionConsumerMemoryDataManager
implements HsDataView {
@GuardedBy("consumerLock")
private final Deque<HsBufferContext> unConsumedBuffers = new
LinkedList<>();
- @GuardedBy("consumerLock")
private final AtomicInteger backlog = new AtomicInteger(0);
Review Comment:
Indeed, this `guardedBy` is required. We have already mark `
@SuppressWarnings("FieldAccessNotGuarded")` for `getBacklog` as it is
unsynchronized by design. All other places should be protected by lock, so I
still think this field is no need atomic.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImpl.java:
##########
@@ -329,13 +327,15 @@ private Optional<BufferIndexOrError>
checkAndGetFirstBufferIndexOrError(
return Optional.ofNullable(peek);
}
- private void increaseBacklog(Buffer buffer) {
+ @GuardedBy("lock")
+ private void tryIncreaseBacklog(Buffer buffer) {
if (buffer.isBuffer()) {
backlog.getAndIncrement();
}
}
- private void decreaseBacklog(Buffer buffer) {
+ @GuardedBy("lock")
Review Comment:
Should be removed as `backlog` is not guarded by `lock`.
--
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]