reswqa commented on code in PR #20371:
URL: https://github.com/apache/flink/pull/20371#discussion_r933093882


##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionView.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import 
org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The read view of HsResultPartition, data can be read from memory or disk. 
*/
+public class HsSubpartitionView
+        implements ResultSubpartitionView, 
HsSubpartitionViewInternalOperations {
+    private final BufferAvailabilityListener availabilityListener;
+    private final Object lock = new Object();
+
+    /** Index of last consumed buffer. */
+    @GuardedBy("lock")
+    private int consumeOffset = -1;
+
+    @GuardedBy("lock")
+    private boolean needNotify = false;
+
+    @Nullable
+    @GuardedBy("lock")
+    private Buffer.DataType cachedNextDataType = null;
+
+    @Nullable
+    @GuardedBy("lock")
+    private Throwable failureCause = null;
+
+    @GuardedBy("lock")
+    private boolean isReleased = false;
+
+    @GuardedBy("lock")
+    private HsDataView diskDataView;
+
+    @GuardedBy("lock")
+    private HsDataView memoryDataView;
+
+    public HsSubpartitionView(BufferAvailabilityListener availabilityListener) 
{
+        this.availabilityListener = availabilityListener;
+    }
+
+    @Nullable
+    @Override
+    public BufferAndBacklog getNextBuffer() {
+        synchronized (lock) {
+            try {
+                checkNotNull(diskDataView, "disk data view must be not null.");
+                checkNotNull(memoryDataView, "memory data view must be not 
null.");
+
+                Optional<BufferAndBacklog> bufferToConsume = tryReadFromDisk();
+                if (!bufferToConsume.isPresent()) {
+                    bufferToConsume = 
memoryDataView.consumeBuffer(consumeOffset + 1);
+                }
+                updateConsumingStatus(bufferToConsume);
+                return bufferToConsume.orElse(null);
+            } catch (Throwable cause) {
+                releaseInternal(cause);
+                return null;
+            }
+        }
+    }
+
+    @Override
+    public void notifyDataAvailable() {
+        boolean notifyDownStream = false;
+        synchronized (lock) {
+            if (isReleased) {
+                return;
+            }
+            if (needNotify) {
+                notifyDownStream = true;
+                needNotify = false;
+            }
+        }
+        // notify outside of lock to avoid deadlock
+        if (notifyDownStream) {
+            availabilityListener.notifyDataAvailable();
+        }
+    }
+
+    @Override
+    public AvailabilityWithBacklog getAvailabilityAndBacklog(int 
numCreditsAvailable) {
+        synchronized (lock) {
+            boolean availability = numCreditsAvailable > 0;
+            if (numCreditsAvailable <= 0
+                    && cachedNextDataType != null
+                    && cachedNextDataType == Buffer.DataType.EVENT_BUFFER) {
+                availability = true;
+            }
+            return new AvailabilityWithBacklog(availability, 
diskDataView.getBacklog());
+        }
+    }
+
+    @Override
+    public void releaseAllResources() throws IOException {
+        releaseInternal(null);
+    }
+
+    @Override
+    public boolean isReleased() {
+        synchronized (lock) {
+            return isReleased;
+        }
+    }
+
+    @Override
+    public int getConsumingOffset() {
+        synchronized (lock) {
+            return consumeOffset;
+        }
+    }
+
+    @Override
+    public Throwable getFailureCause() {
+        synchronized (lock) {
+            return failureCause;
+        }
+    }
+
+    /**
+     * Set {@link HsDataView} for this subpartition, this method only called 
when {@link
+     * HsSubpartitionFileReader} is creating.
+     */
+    void setDiskDataView(HsDataView diskDataView) {
+        synchronized (lock) {
+            checkState(this.diskDataView == null, "repeatedly set disk data 
view is not allowed.");
+            this.diskDataView = diskDataView;
+        }
+    }
+
+    /**
+     * Set {@link HsDataView} for this subpartition, this method only called 
when {@link
+     * HsSubpartitionFileReader} is creating.
+     */
+    void setMemoryDataView(HsDataView memoryDataView) {
+        synchronized (lock) {
+            checkState(
+                    this.memoryDataView == null, "repeatedly set memory data 
view is not allowed.");
+            this.memoryDataView = memoryDataView;
+        }
+    }
+
+    @Override
+    public void resumeConsumption() {
+        throw new UnsupportedOperationException("resumeConsumption should 
never be called.");
+    }
+
+    @Override
+    public void acknowledgeAllDataProcessed() {
+        // in case of bounded partitions there is no upstream to acknowledge, 
we simply ignore
+        // the ack, as there are no checkpoints
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    @Override
+    public int unsynchronizedGetNumberOfQueuedBuffers() {
+        return diskDataView.getBacklog();
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    @Override
+    public int getNumberOfQueuedBuffers() {
+        return diskDataView.getBacklog();
+    }
+
+    @Override
+    public void notifyNewBufferSize(int newBufferSize) {
+        throw new UnsupportedOperationException("Method should never be 
called.");
+    }
+
+    // -------------------------------
+    //       Internal Methods
+    // -------------------------------
+
+    @GuardedBy("lock")
+    private Optional<BufferAndBacklog> tryReadFromDisk() throws Throwable {
+        final int nextBufferIndexToConsume = consumeOffset + 1;
+        Optional<BufferAndBacklog> bufferToConsume =
+                diskDataView.consumeBuffer(nextBufferIndexToConsume);
+        if (!bufferToConsume.isPresent()) {
+            return Optional.empty();
+        }
+        BufferAndBacklog bufferAndBacklog = bufferToConsume.get();
+        if (bufferAndBacklog.getNextDataType() == Buffer.DataType.NONE) {
+            Buffer.DataType dataType =
+                    
memoryDataView.peekNextToConsumeDataType(nextBufferIndexToConsume + 1);
+            return Optional.of(
+                    new BufferAndBacklog(
+                            bufferAndBacklog.buffer(),
+                            bufferAndBacklog.buffersInBacklog(),
+                            dataType,
+                            bufferAndBacklog.getSequenceNumber()));
+        }
+        return bufferToConsume;

Review Comment:
   Apply it.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionView.java:
##########
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.io.network.partition.hybrid;
+
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import 
org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import 
org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+
+import javax.annotation.Nullable;
+import javax.annotation.concurrent.GuardedBy;
+
+import java.io.IOException;
+import java.util.Optional;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** The read view of HsResultPartition, data can be read from memory or disk. 
*/
+public class HsSubpartitionView
+        implements ResultSubpartitionView, 
HsSubpartitionViewInternalOperations {
+    private final BufferAvailabilityListener availabilityListener;
+    private final Object lock = new Object();
+
+    /** Index of last consumed buffer. */
+    @GuardedBy("lock")
+    private int consumeOffset = -1;
+
+    @GuardedBy("lock")
+    private boolean needNotify = false;
+
+    @Nullable
+    @GuardedBy("lock")
+    private Buffer.DataType cachedNextDataType = null;
+
+    @Nullable
+    @GuardedBy("lock")
+    private Throwable failureCause = null;
+
+    @GuardedBy("lock")
+    private boolean isReleased = false;
+
+    @GuardedBy("lock")
+    private HsDataView diskDataView;
+
+    @GuardedBy("lock")
+    private HsDataView memoryDataView;
+
+    public HsSubpartitionView(BufferAvailabilityListener availabilityListener) 
{
+        this.availabilityListener = availabilityListener;
+    }
+
+    @Nullable
+    @Override
+    public BufferAndBacklog getNextBuffer() {
+        synchronized (lock) {
+            try {
+                checkNotNull(diskDataView, "disk data view must be not null.");
+                checkNotNull(memoryDataView, "memory data view must be not 
null.");
+
+                Optional<BufferAndBacklog> bufferToConsume = tryReadFromDisk();
+                if (!bufferToConsume.isPresent()) {
+                    bufferToConsume = 
memoryDataView.consumeBuffer(consumeOffset + 1);
+                }
+                updateConsumingStatus(bufferToConsume);
+                return bufferToConsume.orElse(null);
+            } catch (Throwable cause) {
+                releaseInternal(cause);
+                return null;
+            }
+        }
+    }
+
+    @Override
+    public void notifyDataAvailable() {
+        boolean notifyDownStream = false;
+        synchronized (lock) {
+            if (isReleased) {
+                return;
+            }
+            if (needNotify) {
+                notifyDownStream = true;
+                needNotify = false;
+            }
+        }
+        // notify outside of lock to avoid deadlock
+        if (notifyDownStream) {
+            availabilityListener.notifyDataAvailable();
+        }
+    }
+
+    @Override
+    public AvailabilityWithBacklog getAvailabilityAndBacklog(int 
numCreditsAvailable) {
+        synchronized (lock) {
+            boolean availability = numCreditsAvailable > 0;
+            if (numCreditsAvailable <= 0
+                    && cachedNextDataType != null
+                    && cachedNextDataType == Buffer.DataType.EVENT_BUFFER) {
+                availability = true;
+            }
+            return new AvailabilityWithBacklog(availability, 
diskDataView.getBacklog());
+        }
+    }
+
+    @Override
+    public void releaseAllResources() throws IOException {
+        releaseInternal(null);
+    }
+
+    @Override
+    public boolean isReleased() {
+        synchronized (lock) {
+            return isReleased;
+        }
+    }
+
+    @Override
+    public int getConsumingOffset() {
+        synchronized (lock) {
+            return consumeOffset;
+        }
+    }
+
+    @Override
+    public Throwable getFailureCause() {
+        synchronized (lock) {
+            return failureCause;
+        }
+    }
+
+    /**
+     * Set {@link HsDataView} for this subpartition, this method only called 
when {@link
+     * HsSubpartitionFileReader} is creating.
+     */
+    void setDiskDataView(HsDataView diskDataView) {
+        synchronized (lock) {
+            checkState(this.diskDataView == null, "repeatedly set disk data 
view is not allowed.");
+            this.diskDataView = diskDataView;
+        }
+    }
+
+    /**
+     * Set {@link HsDataView} for this subpartition, this method only called 
when {@link
+     * HsSubpartitionFileReader} is creating.
+     */
+    void setMemoryDataView(HsDataView memoryDataView) {
+        synchronized (lock) {
+            checkState(
+                    this.memoryDataView == null, "repeatedly set memory data 
view is not allowed.");
+            this.memoryDataView = memoryDataView;
+        }
+    }
+
+    @Override
+    public void resumeConsumption() {
+        throw new UnsupportedOperationException("resumeConsumption should 
never be called.");
+    }
+
+    @Override
+    public void acknowledgeAllDataProcessed() {
+        // in case of bounded partitions there is no upstream to acknowledge, 
we simply ignore
+        // the ack, as there are no checkpoints
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    @Override
+    public int unsynchronizedGetNumberOfQueuedBuffers() {
+        return diskDataView.getBacklog();
+    }
+
+    @SuppressWarnings("FieldAccessNotGuarded")
+    @Override
+    public int getNumberOfQueuedBuffers() {
+        return diskDataView.getBacklog();
+    }
+
+    @Override
+    public void notifyNewBufferSize(int newBufferSize) {
+        throw new UnsupportedOperationException("Method should never be 
called.");
+    }
+
+    // -------------------------------
+    //       Internal Methods
+    // -------------------------------
+
+    @GuardedBy("lock")
+    private Optional<BufferAndBacklog> tryReadFromDisk() throws Throwable {
+        final int nextBufferIndexToConsume = consumeOffset + 1;
+        Optional<BufferAndBacklog> bufferToConsume =
+                diskDataView.consumeBuffer(nextBufferIndexToConsume);
+        if (!bufferToConsume.isPresent()) {
+            return Optional.empty();
+        }
+        BufferAndBacklog bufferAndBacklog = bufferToConsume.get();
+        if (bufferAndBacklog.getNextDataType() == Buffer.DataType.NONE) {
+            Buffer.DataType dataType =
+                    
memoryDataView.peekNextToConsumeDataType(nextBufferIndexToConsume + 1);
+            return Optional.of(
+                    new BufferAndBacklog(
+                            bufferAndBacklog.buffer(),
+                            bufferAndBacklog.buffersInBacklog(),
+                            dataType,
+                            bufferAndBacklog.getSequenceNumber()));
+        }
+        return bufferToConsume;

Review Comment:
   Apply this change.



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