Skip to content

Commit

Permalink
[FLINK-8733][network] fix SpillableSubpartition#spillFinishedBufferCo…
Browse files Browse the repository at this point in the history
…nsumers() not counting spilled bytes

This closes apache#5549.
  • Loading branch information
Nico Kruber authored and tillrohrmann committed Feb 27, 2018
1 parent 915213c commit 4bf76ae
Show file tree
Hide file tree
Showing 2 changed files with 25 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

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

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.runtime.io.disk.iomanager.BufferFileWriter;
import org.apache.flink.runtime.io.disk.iomanager.IOManager;
Expand Down Expand Up @@ -240,13 +241,15 @@ public int releaseMemory() throws IOException {
return 0;
}

private long spillFinishedBufferConsumers() throws IOException {
@VisibleForTesting
protected long spillFinishedBufferConsumers() throws IOException {
long spilledBytes = 0;

while (!buffers.isEmpty()) {
BufferConsumer bufferConsumer = buffers.peek();
Buffer buffer = bufferConsumer.build();
updateStatistics(buffer);
spilledBytes += buffer.getSize();
spillWriter.writeBlock(buffer);

if (bufferConsumer.isFinished()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsyncWithNoOpBufferFileWriter;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
import org.apache.flink.runtime.io.network.buffer.BufferBuilder;
import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils;
import org.apache.flink.runtime.io.network.buffer.BufferConsumer;
import org.apache.flink.runtime.io.network.buffer.BufferProvider;
Expand All @@ -40,12 +41,14 @@
import org.mockito.stubbing.Answer;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;

import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createBufferBuilder;
import static org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils.createFilledBufferConsumer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
Expand Down Expand Up @@ -700,6 +703,24 @@ private void testCleanupReleasedPartition(boolean spilled, boolean createView) t
// assertEquals((createView ? 4 : 0) + 2 * BUFFER_DATA_SIZE, partition.getTotalNumberOfBytes());
}

/**
* Tests {@link SpillableSubpartition#spillFinishedBufferConsumers()} spilled bytes counting.
*/
@Test
public void testSpillFinishedBufferConsumers() throws Exception {
SpillableSubpartition partition = createSubpartition();
BufferBuilder bufferBuilder = createBufferBuilder(BUFFER_DATA_SIZE);

try (BufferConsumer buffer = bufferBuilder.createBufferConsumer()) {
partition.add(buffer);
assertEquals(0, partition.releaseMemory());
// finally fill the buffer with some bytes
bufferBuilder.appendAndCommit(ByteBuffer.allocate(BUFFER_DATA_SIZE));
bufferBuilder.finish(); // so that this buffer can be removed from the queue
assertEquals(BUFFER_DATA_SIZE, partition.spillFinishedBufferConsumers());
}
}

/**
* An {@link IOManagerAsync} that creates closed {@link BufferFileWriter} instances in its
* {@link #createBufferFileWriter(FileIOChannel.ID)} method.
Expand Down

0 comments on commit 4bf76ae

Please sign in to comment.