xintongsong commented on code in PR #20371:
URL: https://github.com/apache/flink/pull/20371#discussion_r931891944
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsBufferContext.java:
##########
@@ -97,24 +97,34 @@ public Optional<CompletableFuture<Void>> getSpilledFuture()
{
return Optional.ofNullable(spilledFuture);
}
+ /** Mark buffer status to release. */
public void release() {
- checkState(!released, "Release buffer repeatedly is unexpected.");
+ if (isReleased()) {
+ return;
+ }
released = true;
// decrease ref count when buffer is released from memory.
buffer.recycleBuffer();
}
- public void startSpilling(CompletableFuture<Void> spilledFuture) {
- checkState(!released, "Buffer is already released.");
- checkState(
- !spillStarted && this.spilledFuture == null,
- "Spill buffer repeatedly is unexpected.");
+ /**
+ * Mark buffer status to startSpilling.
+ *
+ * @param spilledFuture completable future of this buffer's spilling
operation.
+ * @return {@link Optional#empty()}, if spilling of the buffer has been
started before or the
+ * buffer has been released already; Optional of this buffer context,
otherwise.
+ */
+ public Optional<HsBufferContext> startSpilling(CompletableFuture<Void>
spilledFuture) {
Review Comment:
It doesn't make sense this method returns itself as an optional.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -106,6 +107,30 @@ public void append(ByteBuffer record, int targetChannel,
Buffer.DataType dataTyp
}
}
+ /** Close this {@link HsMemoryDataManager}, it means no data can append to
memory. */
+ public void close() {
+ Decision decision = callWithLock(() ->
spillStrategy.onResultPartitionClosed(this));
+ handleDecision(Optional.of(decision));
+ spiller.close();
+ }
+
+ /**
+ * Release this {@link HsMemoryDataManager}, it means all memory taken by
this class will
+ * recycle.
+ */
+ public void release() {
+ spiller.release();
+ }
+
+ /**
+ * Setup components of {@link HsMemoryDataManager}.
+ *
+ * <p>This method only called by main task thread.
+ */
+ public void setup() throws IOException {
+ spiller.setup();
+ }
Review Comment:
If we are creating `HsMemoryDataManager` in `ResultPartition#setup` anyway,
we should not need this method, and the relevant changes to the spiller.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsDiskDataView.java:
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.partition.ResultSubpartition;
+
+/**
+ * A view for {@link HsSubpartitionView} to find out what data exists in disk
and polling the data.
+ */
+public interface HsDiskDataView {
Review Comment:
```suggestion
public interface HsFileDataView {
```
--
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]