xintongsong commented on code in PR #20371:
URL: https://github.com/apache/flink/pull/20371#discussion_r932828188
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -68,10 +70,16 @@ public HsMemoryDataManager(
BufferPool bufferPool,
HsSpillingStrategy spillStrategy,
HsFileDataIndex fileDataIndex,
- FileChannel dataFileChannel) {
+ Path dataFilePath)
+ throws IOException {
this.numSubpartitions = numSubpartitions;
this.bufferPool = bufferPool;
- this.spiller = new HsMemoryDataSpiller(dataFileChannel);
+ this.spiller =
+ new HsMemoryDataSpiller(
+ FileChannel.open(
+ dataFilePath,
+ StandardOpenOption.CREATE_NEW,
+ StandardOpenOption.WRITE));
Review Comment:
I'd suggest to move opening of the file channel into the spiller, so that
the spiller is responsible for both opening and closing the file channel.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionMemoryDataManager.java:
##########
@@ -145,8 +146,19 @@ public Optional<HsMemoryDataManager.BufferAndNextDataType>
consumeBuffer(int toC
tuple.f0.getBufferIndexAndChannel()));
return bufferAndNextDataType.map(
tuple ->
- new HsMemoryDataManager.BufferAndNextDataType(
- tuple.f0.getBuffer(), tuple.f1));
+ new BufferAndBacklog(
+ tuple.f0.getBuffer(), getBacklog(), tuple.f1,
toConsumeIndex));
+ }
+
+ // Get backlog of memory data, always return 0.
Review Comment:
Move to inside the method. And explain why.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionMemoryDataManager.java:
##########
@@ -354,8 +354,7 @@ private void addFinishedBuffer(HsBufferContext
bufferContext) {
return unConsumedBuffers.isEmpty();
Review Comment:
Shouldn't this be `unConsumedBuffers.size() <= 1`?
##########
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:
```suggestion
final int nextBufferIndexToConsume = consumeOffset + 1;
return diskDataView
.consumeBuffer(nextBufferIndexToConsume)
.map(
bufferAndBacklog -> {
if (bufferAndBacklog.getNextDataType() ==
Buffer.DataType.NONE) {
return new BufferAndBacklog(
bufferAndBacklog.buffer(),
bufferAndBacklog.buffersInBacklog(),
memoryDataView.peekNextToConsumeDataType(
nextBufferIndexToConsume +
1),
bufferAndBacklog.getSequenceNumber());
}
return bufferAndBacklog;
});```
##########
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;
Review Comment:
Why not just name this `lastConsumedBufferIndex`
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsMemoryDataManager.java:
##########
@@ -115,6 +123,22 @@ public void append(ByteBuffer record, int targetChannel,
Buffer.DataType dataTyp
}
}
+ /**
+ * Register {@link HsSubpartitionViewInternalOperations} to {@link
+ * #subpartitionViewOperationsMap}. It is used to obtain the consumption
progress of the
+ * subpartition.
+ */
+ public void registerSubpartitionView(
+ int subpartitionId, HsSubpartitionViewInternalOperations
viewOperations) {
+ HsSubpartitionViewInternalOperations oldView =
+ subpartitionViewOperationsMap.put(subpartitionId,
viewOperations);
+ if (oldView != null) {
+ LOG.debug(
+ "subpartition : {} register subpartition view will replace
old view. ",
+ subpartitionId);
+ }
Review Comment:
Do we need notify downstream if there's already data for the subpartition?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsFileDataManager.java:
##########
@@ -176,6 +177,19 @@ public void deleteShuffleFile() {
IOUtils.deleteFileQuietly(dataFilePath);
}
+ /**
+ * Release specific {@link HsSubpartitionFileReader} from {@link
HsFileDataManager}.
+ *
+ * @param subpartitionFileReader to release.
+ */
+ public void releaseSubpartitionReader(HsSubpartitionFileReader
subpartitionFileReader) {
+ synchronized (lock) {
+ if (allReaders.contains(subpartitionFileReader)) {
+ failedReaders.add(subpartitionFileReader);
Review Comment:
I think we need to call `removeFailedReaders()` here. Otherwise, in the next
round of reading, `prepareForScheduling` can be called on this released reader.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionMemoryDataManager.java:
##########
@@ -114,16 +115,16 @@ public DataType peekNextToConsumeDataType(int
nextToConsumeIndex) {
/**
* Check whether the head of {@link #unConsumedBuffers} is the buffer to
be consumed. If so,
- * return the buffer and next data type.
+ * return the buffer and backlog.
*
* @param toConsumeIndex index of buffer to be consumed.
* @return If the head of {@link #unConsumedBuffers} is target, return
optional of the buffer
- * and next data type. Otherwise, return {@link Optional#empty()}.
+ * and backlog. Otherwise, return {@link Optional#empty()}.
*/
@SuppressWarnings("FieldAccessNotGuarded")
// Note that: callWithLock ensure that code block guarded by
resultPartitionReadLock and
// subpartitionLock.
- public Optional<HsMemoryDataManager.BufferAndNextDataType>
consumeBuffer(int toConsumeIndex) {
+ public Optional<BufferAndBacklog> consumeBuffer(int toConsumeIndex) {
Review Comment:
`@Override`
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImpl.java:
##########
@@ -192,6 +198,50 @@ public Deque<BufferIndexOrError> getLoadedBuffers() {
return loadedBuffers;
}
+ @Override
+ public Optional<ResultSubpartition.BufferAndBacklog> consumeBuffer(int
nextBufferToConsume)
+ throws Throwable {
+ BufferIndexOrError current = loadedBuffers.poll();
+
+ if (current == null) {
+ return Optional.empty();
+ }
+
+ if (current.getThrowable().isPresent()) {
+ throw current.getThrowable().get();
+ }
+
+ BufferIndexOrError next = loadedBuffers.peek();
+
+ Buffer.DataType nextDataType = next == null ? Buffer.DataType.NONE :
next.getDataType();
+ int backlog = loadedBuffers.size();
+ int bufferIndex = current.getIndex();
+ Buffer buffer =
+ current.getBuffer()
+ .orElseThrow(
+ () ->
+ new NullPointerException(
+ "Get a non-throwable and
non-buffer bufferIndexOrError, which is not allowed"));
+ return Optional.of(
+ ResultSubpartition.BufferAndBacklog.fromBufferAndLookahead(
+ buffer, nextDataType, backlog, bufferIndex));
+ }
+
+ @Override
+ public Buffer.DataType peekNextToConsumeDataType(int nextBufferToConsume) {
+ return Buffer.DataType.NONE;
Review Comment:
We should still check the `loadedBuffers` here.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionFileReaderImpl.java:
##########
@@ -192,6 +198,50 @@ public Deque<BufferIndexOrError> getLoadedBuffers() {
return loadedBuffers;
}
+ @Override
+ public Optional<ResultSubpartition.BufferAndBacklog> consumeBuffer(int
nextBufferToConsume)
+ throws Throwable {
+ BufferIndexOrError current = loadedBuffers.poll();
Review Comment:
Shouldn't we check the `nextBufferToConsume` before polling?
##########
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;
+ }
+
+ @SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+ @GuardedBy("lock")
+ private void updateConsumingStatus(Optional<BufferAndBacklog>
bufferAndBacklog) {
+ assert Thread.holdsLock(lock);
+ // if consumed, update and check consume offset
+ if (bufferAndBacklog.isPresent()) {
+ ++consumeOffset;
+ checkState(bufferAndBacklog.get().getSequenceNumber() ==
consumeOffset);
+ }
+
+ // update need-notify
+ needNotify =
!(bufferAndBacklog.map(BufferAndBacklog::isDataAvailable).orElse(false));
Review Comment:
```suggestion
boolean dataAvailable =
bufferAndBacklog.map(BufferAndBacklog::isDataAvailable).orElse(false);
needNotify = !dataAvailable;
```
##########
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;
Review Comment:
Mark `@Nullable`, and comment it can be null only before initialization.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartition.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool;
+import org.apache.flink.runtime.io.network.api.EndOfData;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.StopMode;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import
org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link HsResultPartition} appends records and events to {@link
HsMemoryDataManager}, the shuffle
+ * data maybe spilled to disk according to the {@link HsSpillingStrategy}, and
the downstream can
+ * consume data from memory or disk.
+ */
+public class HsResultPartition extends ResultPartition {
+ public static final String DATA_FILE_SUFFIX = ".hybrid.data";
+
+ private final HsFileDataIndex dataIndex;
+
+ private final HsFileDataManager fileDataManager;
+
+ private final Path dataFilePath;
+
+ private final int networkBufferSize;
+
+ private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+ private boolean hasNotifiedEndOfUserRecords;
+
+ private HsMemoryDataManager memoryDataManager;
Review Comment:
```suggestion
@Nullable
private HsMemoryDataManager memoryDataManager;
```
##########
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionViewTest.java:
##########
@@ -0,0 +1,319 @@
+/*
+ * 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.buffer.Buffer.DataType;
+import
org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import
org.apache.flink.runtime.io.network.partition.NoOpBufferAvailablityListener;
+import
org.apache.flink.runtime.io.network.partition.ResultSubpartition.BufferAndBacklog;
+import
org.apache.flink.runtime.io.network.partition.ResultSubpartitionView.AvailabilityWithBacklog;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Tests for {@link HsSubpartitionView}. */
+class HsSubpartitionViewTest {
+ @Test
+ void testGetNextBufferFromDisk() {
+ HsSubpartitionView subpartitionView = createSubpartitionView();
+
+ BufferAndBacklog bufferAndBacklog = createBufferAndBacklog(0,
DataType.DATA_BUFFER, 0);
+ CompletableFuture<Void> consumeBufferFromMemoryFuture = new
CompletableFuture<>();
+ TestingHsDataView diskDataView =
+ TestingHsDataView.builder()
+ .setConsumeBufferFunction(
+ (bufferToConsume) ->
Optional.of(bufferAndBacklog))
+ .build();
+ TestingHsDataView memoryDataView =
+ TestingHsDataView.builder()
+ .setConsumeBufferFunction(
+ (ignore) -> {
+
consumeBufferFromMemoryFuture.complete(null);
+ return Optional.empty();
+ })
+ .build();
+ subpartitionView.setDiskDataView(diskDataView);
+ subpartitionView.setMemoryDataView(memoryDataView);
+
+ BufferAndBacklog nextBuffer = subpartitionView.getNextBuffer();
+ assertThat(consumeBufferFromMemoryFuture).isNotCompleted();
+ assertThat(nextBuffer).isSameAs(bufferAndBacklog);
+ }
+
+ @Test
+ void testGetNextBufferFromDiskNextDataTypeIsNone() {
+ HsSubpartitionView subpartitionView = createSubpartitionView();
+ BufferAndBacklog bufferAndBacklog = createBufferAndBacklog(0,
DataType.NONE, 0);
+ CompletableFuture<Void> peekNextDataTypeFuture = new
CompletableFuture<>();
+
+ TestingHsDataView diskDataView =
+ TestingHsDataView.builder()
+ .setConsumeBufferFunction(
+ (bufferToConsume) ->
Optional.of(bufferAndBacklog))
+ .build();
+
+ TestingHsDataView memoryDataView =
+ TestingHsDataView.builder()
+ .setPeekNextToConsumeDataTypeFunction(
+ (bufferToConsume) -> {
+ assertThat(bufferToConsume).isEqualTo(1);
+ peekNextDataTypeFuture.complete(null);
+ return DataType.EVENT_BUFFER;
+ })
+ .build();
+ subpartitionView.setDiskDataView(diskDataView);
+ subpartitionView.setMemoryDataView(memoryDataView);
+
+ BufferAndBacklog nextBuffer = subpartitionView.getNextBuffer();
+ assertThat(nextBuffer).isNotNull();
+ assertThat(peekNextDataTypeFuture).isCompleted();
+ assertThat(nextBuffer.buffer()).isSameAs(bufferAndBacklog.buffer());
+
assertThat(nextBuffer.buffersInBacklog()).isEqualTo(bufferAndBacklog.buffersInBacklog());
+
assertThat(nextBuffer.getSequenceNumber()).isEqualTo(bufferAndBacklog.getSequenceNumber());
+
assertThat(nextBuffer.getNextDataType()).isEqualTo(DataType.EVENT_BUFFER);
+ }
+
+ @Test
+ void testGetNextBufferFromMemory() {
+ HsSubpartitionView subpartitionView = createSubpartitionView();
+
+ BufferAndBacklog bufferAndBacklog = createBufferAndBacklog(0,
DataType.DATA_BUFFER, 0);
+ TestingHsDataView memoryDataView =
+ TestingHsDataView.builder()
+ .setConsumeBufferFunction(
+ (bufferToConsume) ->
Optional.of(bufferAndBacklog))
+ .build();
+ TestingHsDataView diskDataView =
+ TestingHsDataView.builder()
+ .setConsumeBufferFunction((bufferToConsume) ->
Optional.empty())
+ .build();
+ subpartitionView.setDiskDataView(diskDataView);
+ subpartitionView.setMemoryDataView(memoryDataView);
+
+ BufferAndBacklog nextBuffer = subpartitionView.getNextBuffer();
+ assertThat(nextBuffer).isSameAs(bufferAndBacklog);
+ }
+
+ @Test
+ void testGetNextDataTypeWrongSequenceNumber() {}
Review Comment:
Empty test case.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartition.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool;
+import org.apache.flink.runtime.io.network.api.EndOfData;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.StopMode;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import
org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link HsResultPartition} appends records and events to {@link
HsMemoryDataManager}, the shuffle
+ * data maybe spilled to disk according to the {@link HsSpillingStrategy}, and
the downstream can
+ * consume data from memory or disk.
+ */
+public class HsResultPartition extends ResultPartition {
+ public static final String DATA_FILE_SUFFIX = ".hybrid.data";
+
+ private final HsFileDataIndex dataIndex;
+
+ private final HsFileDataManager fileDataManager;
+
+ private final Path dataFilePath;
+
+ private final int networkBufferSize;
+
+ private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+ private boolean hasNotifiedEndOfUserRecords;
+
+ private HsMemoryDataManager memoryDataManager;
+
+ public HsResultPartition(
+ String owningTaskName,
+ int partitionIndex,
+ ResultPartitionID partitionId,
+ ResultPartitionType partitionType,
+ int numSubpartitions,
+ int numTargetKeyGroups,
+ BatchShuffleReadBufferPool readBufferPool,
+ Executor readIOExecutor,
+ ResultPartitionManager partitionManager,
+ String dataFileBashPath,
+ int networkBufferSize,
+ HybridShuffleConfiguration hybridShuffleConfiguration,
+ @Nullable BufferCompressor bufferCompressor,
+ SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+ super(
+ owningTaskName,
+ partitionIndex,
+ partitionId,
+ partitionType,
+ numSubpartitions,
+ numTargetKeyGroups,
+ partitionManager,
+ bufferCompressor,
+ bufferPoolFactory);
+ this.networkBufferSize = networkBufferSize;
+ this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+ this.dataFilePath = new File(dataFileBashPath +
DATA_FILE_SUFFIX).toPath();
+ this.hybridShuffleConfiguration = hybridShuffleConfiguration;
+ this.fileDataManager =
+ new HsFileDataManager(
+ readBufferPool,
+ readIOExecutor,
+ dataIndex,
+ dataFilePath,
+ HsSubpartitionFileReaderImpl.Factory.INSTANCE,
+ hybridShuffleConfiguration);
+ }
+
+ // Called by task thread.
+ @Override
+ protected void setupInternal() throws IOException {
+ if (isReleased()) {
+ throw new IOException("Result partition has been released.");
+ }
+ this.fileDataManager.setup();
+ this.memoryDataManager =
+ new HsMemoryDataManager(
+ numSubpartitions,
+ networkBufferSize,
+ bufferPool,
+ getSpillingStrategy(hybridShuffleConfiguration),
+ dataIndex,
+ dataFilePath);
+ }
+
+ @Override
+ public void emitRecord(ByteBuffer record, int targetSubpartition) throws
IOException {
+ emit(record, targetSubpartition, Buffer.DataType.DATA_BUFFER);
+ }
+
+ @Override
+ public void broadcastRecord(ByteBuffer record) throws IOException {
+ broadcast(record, Buffer.DataType.DATA_BUFFER);
+ }
+
+ @Override
+ public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent)
throws IOException {
+ Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+ try {
+ ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+ broadcast(serializedEvent, buffer.getDataType());
+ } finally {
+ buffer.recycleBuffer();
+ }
+ }
+
+ private void broadcast(ByteBuffer record, Buffer.DataType dataType) throws
IOException {
+ for (int i = 0; i < numSubpartitions; i++) {
+ emit(record.duplicate(), i, dataType);
+ }
+ }
+
+ private void emit(ByteBuffer record, int targetSubpartition,
Buffer.DataType dataType)
+ throws IOException {
+ checkInProduceState();
+ memoryDataManager.append(record, targetSubpartition, dataType);
+ }
+
+ @Override
+ public ResultSubpartitionView createSubpartitionView(
+ int subpartitionId, BufferAvailabilityListener
availabilityListener)
+ throws IOException {
+ checkState(!isReleased(), "ResultPartition already released.");
+ HsSubpartitionView subpartitionView = new
HsSubpartitionView(availabilityListener);
+ HsDataView diskDataView =
+ fileDataManager.registerNewSubpartition(subpartitionId,
subpartitionView);
+
+ memoryDataManager.registerSubpartitionView(subpartitionId,
subpartitionView);
+ subpartitionView.setDiskDataView(diskDataView);
+ subpartitionView.setMemoryDataView(
+ memoryDataManager.getSubpartitionDataView(subpartitionId));
+ return subpartitionView;
+ }
+
+ @Override
+ public void alignedBarrierTimeout(long checkpointId) throws IOException {
+ // Nothing to do.
+ }
+
+ @Override
+ public void abortCheckpoint(long checkpointId, CheckpointException cause) {
+ // Nothing to do.
+ }
+
+ @Override
+ public void flushAll() {
+ // Nothing to do.
+ }
+
+ @Override
+ public void flush(int subpartitionIndex) {
+ // Nothing to do.
+ }
+
+ @Override
+ public void finish() throws IOException {
+ broadcastEvent(EndOfPartitionEvent.INSTANCE, false);
+
+ checkState(!isReleased(), "Result partition is already released.");
+
+ super.finish();
+ }
+
+ @Override
+ public void close() {
+ // close is called when task is finished or failed.
+ memoryDataManager.close();
+ super.close();
+ }
+
+ @Override
+ protected void releaseInternal() {
+ // release is called when release by scheduler, later than close.
+ // mainly work :
+ // 1. release read scheduler.
+ // 2. delete shuffle file.
+ // 3. release all data in memory.
+
+ // delete the shuffle file only when no reader is reading now.
+ fileDataManager.release().thenRun(fileDataManager::deleteShuffleFile);
Review Comment:
Can we combine `release` and `deleteShuffleFile` into one method?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsSubpartitionMemoryDataManager.java:
##########
@@ -247,6 +261,17 @@ public void
releaseSubpartitionBuffers(List<BufferIndexAndChannel> toRelease) {
}));
}
+ /** Un-synchronized get number of buffers not consumed. */
+ @SuppressWarnings("FieldAccessNotGuarded")
+ public int getNumOfBuffersUnsafe() {
+ return unConsumedBuffers.size();
+ }
+
+ /** Un-synchronized get number of bytes not consumed. */
+ public long getNumOfBytesUnsafe() {
+ return getNumOfBuffersUnsafe() * (long) bufferSize;
+ }
+
Review Comment:
I'm not entirely sure about these calculations. What are the semantics here?
##########
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/hybrid/HsResultPartition.java:
##########
@@ -0,0 +1,261 @@
+/*
+ * 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.configuration.IllegalConfigurationException;
+import org.apache.flink.runtime.checkpoint.CheckpointException;
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.BatchShuffleReadBufferPool;
+import org.apache.flink.runtime.io.network.api.EndOfData;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.StopMode;
+import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.io.network.buffer.BufferCompressor;
+import org.apache.flink.runtime.io.network.buffer.BufferPool;
+import
org.apache.flink.runtime.io.network.partition.BufferAvailabilityListener;
+import org.apache.flink.runtime.io.network.partition.ResultPartition;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionManager;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
+import org.apache.flink.runtime.io.network.partition.ResultSubpartitionView;
+import org.apache.flink.util.function.SupplierWithException;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Path;
+import java.util.concurrent.Executor;
+
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * {@link HsResultPartition} appends records and events to {@link
HsMemoryDataManager}, the shuffle
+ * data maybe spilled to disk according to the {@link HsSpillingStrategy}, and
the downstream can
+ * consume data from memory or disk.
+ */
+public class HsResultPartition extends ResultPartition {
+ public static final String DATA_FILE_SUFFIX = ".hybrid.data";
+
+ private final HsFileDataIndex dataIndex;
+
+ private final HsFileDataManager fileDataManager;
+
+ private final Path dataFilePath;
+
+ private final int networkBufferSize;
+
+ private final HybridShuffleConfiguration hybridShuffleConfiguration;
+
+ private boolean hasNotifiedEndOfUserRecords;
+
+ private HsMemoryDataManager memoryDataManager;
+
+ public HsResultPartition(
+ String owningTaskName,
+ int partitionIndex,
+ ResultPartitionID partitionId,
+ ResultPartitionType partitionType,
+ int numSubpartitions,
+ int numTargetKeyGroups,
+ BatchShuffleReadBufferPool readBufferPool,
+ Executor readIOExecutor,
+ ResultPartitionManager partitionManager,
+ String dataFileBashPath,
+ int networkBufferSize,
+ HybridShuffleConfiguration hybridShuffleConfiguration,
+ @Nullable BufferCompressor bufferCompressor,
+ SupplierWithException<BufferPool, IOException> bufferPoolFactory) {
+ super(
+ owningTaskName,
+ partitionIndex,
+ partitionId,
+ partitionType,
+ numSubpartitions,
+ numTargetKeyGroups,
+ partitionManager,
+ bufferCompressor,
+ bufferPoolFactory);
+ this.networkBufferSize = networkBufferSize;
+ this.dataIndex = new HsFileDataIndexImpl(numSubpartitions);
+ this.dataFilePath = new File(dataFileBashPath +
DATA_FILE_SUFFIX).toPath();
+ this.hybridShuffleConfiguration = hybridShuffleConfiguration;
+ this.fileDataManager =
+ new HsFileDataManager(
+ readBufferPool,
+ readIOExecutor,
+ dataIndex,
+ dataFilePath,
+ HsSubpartitionFileReaderImpl.Factory.INSTANCE,
+ hybridShuffleConfiguration);
+ }
+
+ // Called by task thread.
+ @Override
+ protected void setupInternal() throws IOException {
+ if (isReleased()) {
+ throw new IOException("Result partition has been released.");
+ }
+ this.fileDataManager.setup();
+ this.memoryDataManager =
+ new HsMemoryDataManager(
+ numSubpartitions,
+ networkBufferSize,
+ bufferPool,
+ getSpillingStrategy(hybridShuffleConfiguration),
+ dataIndex,
+ dataFilePath);
+ }
+
+ @Override
+ public void emitRecord(ByteBuffer record, int targetSubpartition) throws
IOException {
+ emit(record, targetSubpartition, Buffer.DataType.DATA_BUFFER);
+ }
+
+ @Override
+ public void broadcastRecord(ByteBuffer record) throws IOException {
+ broadcast(record, Buffer.DataType.DATA_BUFFER);
+ }
+
+ @Override
+ public void broadcastEvent(AbstractEvent event, boolean isPriorityEvent)
throws IOException {
+ Buffer buffer = EventSerializer.toBuffer(event, isPriorityEvent);
+ try {
+ ByteBuffer serializedEvent = buffer.getNioBufferReadable();
+ broadcast(serializedEvent, buffer.getDataType());
+ } finally {
+ buffer.recycleBuffer();
+ }
+ }
+
+ private void broadcast(ByteBuffer record, Buffer.DataType dataType) throws
IOException {
+ for (int i = 0; i < numSubpartitions; i++) {
+ emit(record.duplicate(), i, dataType);
+ }
+ }
+
+ private void emit(ByteBuffer record, int targetSubpartition,
Buffer.DataType dataType)
+ throws IOException {
+ checkInProduceState();
+ memoryDataManager.append(record, targetSubpartition, dataType);
+ }
+
+ @Override
+ public ResultSubpartitionView createSubpartitionView(
+ int subpartitionId, BufferAvailabilityListener
availabilityListener)
+ throws IOException {
+ checkState(!isReleased(), "ResultPartition already released.");
+ HsSubpartitionView subpartitionView = new
HsSubpartitionView(availabilityListener);
+ HsDataView diskDataView =
+ fileDataManager.registerNewSubpartition(subpartitionId,
subpartitionView);
+
+ memoryDataManager.registerSubpartitionView(subpartitionId,
subpartitionView);
+ subpartitionView.setDiskDataView(diskDataView);
+ subpartitionView.setMemoryDataView(
+ memoryDataManager.getSubpartitionDataView(subpartitionId));
Review Comment:
Can me combine `registerSubpartitionView` and `getSubpartitionDataView` into
one method?
--
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]