Skip to content

Commit

Permalink
[FLINK-22424][network] Prevent releasing PipelinedSubpartition while …
Browse files Browse the repository at this point in the history
…Task can still write to it

This bug was happening when a downstream tasks were failing over or being cancelled. If all
of the downstream tasks released subpartition views, underlying memory segments/buffers could
have been recycled, while upstream task was still writting some records.

The problem is fixed by adding the writer (result partition) itself as one more reference
counted user of the result partition
  • Loading branch information
pnowojski authored and TheodoreLx committed Apr 28, 2021
1 parent 5948039 commit 211751c
Show file tree
Hide file tree
Showing 5 changed files with 164 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@
*/
public class PipelinedResultPartition extends BufferWritingResultPartition
implements CheckpointedResultPartition, ChannelStateHolder {
private static final int PIPELINED_RESULT_PARTITION_ITSELF = -42;

/**
* The lock that guard release operations (which can be asynchronously propagated from the
Expand All @@ -64,10 +65,13 @@ public class PipelinedResultPartition extends BufferWritingResultPartition

/**
* The total number of references to subpartitions of this result. The result partition can be
* safely released, iff the reference count is zero.
* safely released, iff the reference count is zero. Every subpartition is an user of the result
* as well the {@link PipelinedResultPartition} is a user itself, as it's writing to those
* results. Even if all consumers are released, partition can not be released until writer
* releases the partition as well.
*/
@GuardedBy("releaseLock")
private int numUnconsumedSubpartitions;
private int numberOfUsers;

public PipelinedResultPartition(
String owningTaskName,
Expand All @@ -92,7 +96,7 @@ public PipelinedResultPartition(
bufferPoolFactory);

this.consumedSubpartitions = new boolean[subpartitions.length];
this.numUnconsumedSubpartitions = subpartitions.length;
this.numberOfUsers = subpartitions.length + 1;
}

@Override
Expand All @@ -110,6 +114,10 @@ public void setChannelStateWriter(ChannelStateWriter channelStateWriter) {
*/
@Override
void onConsumedSubpartition(int subpartitionIndex) {
decrementNumberOfUsers(subpartitionIndex);
}

private void decrementNumberOfUsers(int subpartitionIndex) {
if (isReleased()) {
return;
}
Expand All @@ -119,13 +127,15 @@ void onConsumedSubpartition(int subpartitionIndex) {
// we synchronize only the bookkeeping section, to avoid holding the lock during any
// calls into other components
synchronized (releaseLock) {
if (consumedSubpartitions[subpartitionIndex]) {
// repeated call - ignore
return;
}
if (subpartitionIndex != PIPELINED_RESULT_PARTITION_ITSELF) {
if (consumedSubpartitions[subpartitionIndex]) {
// repeated call - ignore
return;
}

consumedSubpartitions[subpartitionIndex] = true;
remainingUnconsumed = (--numUnconsumedSubpartitions);
consumedSubpartitions[subpartitionIndex] = true;
}
remainingUnconsumed = (--numberOfUsers);
}

LOG.debug(
Expand Down Expand Up @@ -164,7 +174,7 @@ public String toString() {
+ ", "
+ subpartitions.length
+ " subpartitions, "
+ numUnconsumedSubpartitions
+ numberOfUsers
+ " pending consumptions]";
}

Expand All @@ -187,4 +197,10 @@ public void finishReadRecoveredState(boolean notifyAndBlockOnCompletion) throws
.finishReadRecoveredState(notifyAndBlockOnCompletion);
}
}

@Override
public void close() {
decrementNumberOfUsers(PIPELINED_RESULT_PARTITION_ITSELF);
super.close();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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.buffer;

import org.apache.flink.core.memory.MemorySegment;
import org.apache.flink.core.memory.MemorySegmentFactory;

import java.util.concurrent.CompletableFuture;

/** Implementation of {@link BufferPool} that works on un-pooled memory segments. */
public class UnpooledBufferPool implements BufferPool {

private static final int SEGMENT_SIZE = 1024;

@Override
public void lazyDestroy() {}

@Override
public Buffer requestBuffer() {
return new NetworkBuffer(requestMemorySegment(), this);
}

@Override
public BufferBuilder requestBufferBuilder() {
return new BufferBuilder(requestMemorySegment(), this);
}

private MemorySegment requestMemorySegment() {
return MemorySegmentFactory.allocateUnpooledOffHeapMemory(SEGMENT_SIZE, null);
}

@Override
public BufferBuilder requestBufferBuilderBlocking() throws InterruptedException {
return requestBufferBuilder();
}

@Override
public BufferBuilder requestBufferBuilder(int targetChannel) {
return requestBufferBuilder();
}

@Override
public BufferBuilder requestBufferBuilderBlocking(int targetChannel)
throws InterruptedException {
return requestBufferBuilder();
}

@Override
public boolean addBufferListener(BufferListener listener) {
throw new UnsupportedOperationException();
}

@Override
public boolean isDestroyed() {
throw new UnsupportedOperationException();
}

@Override
public int getNumberOfRequiredMemorySegments() {
return Integer.MAX_VALUE;
}

@Override
public int getMaxNumberOfMemorySegments() {
return -1;
}

@Override
public int getNumBuffers() {
return Integer.MAX_VALUE;
}

@Override
public void setNumBuffers(int numBuffers) {
throw new UnsupportedOperationException();
}

@Override
public int getNumberOfAvailableMemorySegments() {
return Integer.MAX_VALUE;
}

@Override
public int bestEffortGetNumOfUsedBuffers() {
return 0;
}

@Override
public void recycle(MemorySegment memorySegment) {
memorySegment.free();
}

@Override
public CompletableFuture<?> getAvailableFuture() {
return AVAILABLE;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,14 @@

package org.apache.flink.runtime.io.network.partition;

import org.apache.flink.runtime.io.network.buffer.UnpooledBufferPool;
import org.apache.flink.util.TestLogger;

import org.junit.Test;

import java.io.IOException;
import java.nio.ByteBuffer;

import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;

Expand All @@ -43,6 +47,25 @@ public void testConsumptionBasedPartitionRelease() {
assertFalse(partition.isReleased());

partition.onConsumedSubpartition(1);
partition.close();
assertTrue(partition.isReleased());
}

@Test
public void testConsumptionBeforePartitionClose() throws IOException {
final ResultPartition partition =
new ResultPartitionBuilder()
.setResultPartitionType(ResultPartitionType.PIPELINED)
.setNumberOfSubpartitions(1)
.setBufferPoolFactory(UnpooledBufferPool::new)
.build();

partition.setup();
partition.emitRecord(ByteBuffer.allocate(16), 0);
partition.onConsumedSubpartition(0);
assertFalse(partition.isReleased());
partition.emitRecord(ByteBuffer.allocate(16), 0);
partition.close();
assertTrue(partition.isReleased());
}

Expand Down Expand Up @@ -77,6 +100,7 @@ public void testReleaseAfterIdempotentCalls() {
partition.onConsumedSubpartition(0);
partition.onConsumedSubpartition(0);
partition.onConsumedSubpartition(1);
partition.close();

assertTrue(partition.isReleased());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@ public void testAddTwoNonFinishedBuffer() throws IOException {
@Test
public void testRelease() {
readView.releaseAllResources();
resultPartition.close();
assertFalse(
resultPartition
.getPartitionManager()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,16 +82,6 @@ public void testSortMergePartitionCreated() {
assertTrue(resultPartition instanceof SortMergeResultPartition);
}

@Test
public void testReleaseOnConsumptionForPipelinedPartition() {
final ResultPartition resultPartition =
createResultPartition(ResultPartitionType.PIPELINED);

resultPartition.onConsumedSubpartition(0);

assertTrue(resultPartition.isReleased());
}

@Test
public void testNoReleaseOnConsumptionForBoundedBlockingPartition() {
final ResultPartition resultPartition = createResultPartition(ResultPartitionType.BLOCKING);
Expand Down

0 comments on commit 211751c

Please sign in to comment.