flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-7499) double buffer release in SpillableSubpartitionView
Date Thu, 23 Nov 2017 17:18:00 GMT

    [ https://issues.apache.org/jira/browse/FLINK-7499?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16264629#comment-16264629
] 

ASF GitHub Bot commented on FLINK-7499:
---------------------------------------

Github user NicoK commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4581#discussion_r152849820
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
---
    @@ -300,6 +315,230 @@ public void testConsumeSpillablePartitionSpilledDuringConsume()
throws Exception
     		assertTrue(buffer.isRecycled());
     	}
     
    +	/**
    +	 * Tests {@link SpillableSubpartition#add(Buffer)} with a spillable finished partition.
    +	 */
    +	@Test
    +	public void testAddOnFinishedSpillablePartition() throws Exception {
    +		SpillableSubpartition partition = createSubpartition();
    +		partition.finish();
    +
    +		Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096),
    +			FreeingBufferRecycler.INSTANCE);
    +		try {
    +			partition.add(buffer);
    +		} finally {
    +			if (!buffer.isRecycled()) {
    +				Assert.fail("buffer not recycled");
    +				buffer.recycle();
    +			}
    +			// finish adds an EndOfPartitionEvent
    +			assertEquals(1, partition.getTotalNumberOfBuffers());
    +			assertEquals(4, partition.getTotalNumberOfBytes());
    +		}
    +	}
    +
    +	/**
    +	 * Tests {@link SpillableSubpartition#add(Buffer)} with a spilled finished partition.
    +	 */
    +	@Test
    +	public void testAddOnFinishedSpilledPartition() throws Exception {
    +		SpillableSubpartition partition = createSubpartition();
    +		assertEquals(0, partition.releaseMemory());
    +		partition.finish();
    +
    +		Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096),
    +			FreeingBufferRecycler.INSTANCE);
    +		try {
    +			partition.add(buffer);
    +		} finally {
    +			if (!buffer.isRecycled()) {
    +				Assert.fail("buffer not recycled");
    +				buffer.recycle();
    +			}
    +			// finish adds an EndOfPartitionEvent
    +			assertEquals(1, partition.getTotalNumberOfBuffers());
    +			assertEquals(4, partition.getTotalNumberOfBytes());
    +		}
    +	}
    +
    +	/**
    +	 * Tests {@link SpillableSubpartition#add(Buffer)} with a spillable released partition.
    +	 */
    +	@Test
    +	public void testAddOnReleasedSpillablePartition() throws Exception {
    +		SpillableSubpartition partition = createSubpartition();
    +		partition.release();
    +
    +		Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096),
    +			FreeingBufferRecycler.INSTANCE);
    +		try {
    +			partition.add(buffer);
    +		} finally {
    +			if (!buffer.isRecycled()) {
    +				Assert.fail("buffer not recycled");
    +				buffer.recycle();
    +			}
    +			assertEquals(0, partition.getTotalNumberOfBuffers());
    +			assertEquals(0, partition.getTotalNumberOfBytes());
    +		}
    +	}
    +
    +	/**
    +	 * Tests {@link SpillableSubpartition#add(Buffer)} with a spilled released partition.
    +	 */
    +	@Test
    +	public void testAddOnReleasedSpilledPartition() throws Exception {
    +		SpillableSubpartition partition = createSubpartition();
    +		partition.release();
    +		assertEquals(0, partition.releaseMemory());
    +
    +		Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096),
    +			FreeingBufferRecycler.INSTANCE);
    +		try {
    +			partition.add(buffer);
    +		} finally {
    +			if (!buffer.isRecycled()) {
    +				Assert.fail("buffer not recycled");
    +				buffer.recycle();
    +			}
    +			assertEquals(0, partition.getTotalNumberOfBuffers());
    +			assertEquals(0, partition.getTotalNumberOfBytes());
    +		}
    +	}
    +
    +	/**
    +	 * Tests {@link SpillableSubpartition#add(Buffer)} with a spilled partition where adding
the
    +	 * write request fails with an exception.
    +	 */
    +	@Test
    +	public void testAddOnSpilledPartitionWithSlowWriter() throws Exception {
    +		IOManager ioManager = new IOManagerAsyncWithStallingBufferFileWriter();
    +		SpillableSubpartition partition = createSubpartition(ioManager);
    +		assertEquals(0, partition.releaseMemory());
    +
    +		Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(4096),
    +			FreeingBufferRecycler.INSTANCE);
    +		try {
    +			partition.add(buffer);
    +		} finally {
    +			ioManager.shutdown();
    --- End diff --
    
    The `ioManager` runs in parallel - we always want to shut it down, even if the checks
fail. Since both our writer implementations don't do anything (`IOManagerAsyncWithStallingBufferFileWriter`
returns a writer that ignores any write requests, `IOManagerAsyncWithClosedBufferFileWriter`
creates a writer that would always throw when executing write requests), it's fine the way
it is.


> double buffer release in SpillableSubpartitionView
> --------------------------------------------------
>
>                 Key: FLINK-7499
>                 URL: https://issues.apache.org/jira/browse/FLINK-7499
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Network
>    Affects Versions: 1.2.0, 1.1.4, 1.3.0, 1.1.5, 1.2.1, 1.2.2, 1.3.1, 1.4.0, 1.3.2, 1.3.3
>            Reporter: Nico Kruber
>            Assignee: Nico Kruber
>            Priority: Blocker
>
> {{SpillableSubpartitionView#releaseMemory()}} recycles its buffers twice: once asynchronously
after the write operation and once in {{SpillableSubpartitionView#releaseMemory()}} after
adding the write operation to the queue.
> 1) if {{SpillableSubpartitionView#releaseMemory()}} hits first and the buffer is recycled,
the memory region may already be reused despite the pending write
> 2) If, for some reason (probably only in tests like {{SpillableSubpartitionTest#testConsumeSpillablePartitionSpilledDuringConsume()}}?),
the buffer is retained and to be used in parallel somewhere else it may also not be available
anymore or contain corrupt data.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message