[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-23 Thread NicoK
Github user NicoK commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r152813899
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
Buffer read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
// End of partition
read = reader.getNextBuffer();
assertNotNull(read);
assertEquals(EndOfPartitionEvent.class, 
EventSerializer.fromBuffer(read, 
ClassLoader.getSystemClassLoader()).getClass());
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
+
+   // finally check that the buffer has been freed after a 
successful (or failed) write
--- End diff --

no - this is why I added numerous more tests now :) thanks for pointing 
this out


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-23 Thread NicoK
Github user NicoK commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r152813796
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
Buffer read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
--- End diff --

alright...extracted those changes into a separate hotfix


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-23 Thread NicoK
Github user NicoK commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r152813679
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
--- End diff --

done


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-23 Thread NicoK
Github user NicoK commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r152813605
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
Buffer read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
// End of partition
read = reader.getNextBuffer();
assertNotNull(read);
assertEquals(EndOfPartitionEvent.class, 
EventSerializer.fromBuffer(read, 
ClassLoader.getSystemClassLoader()).getClass());
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
+
+   // finally check that the buffer has been freed after a 
successful (or failed) write
+   final long deadline = System.currentTimeMillis() + 30_000L; // 
30 secs
+   while (!buffer.isRecycled() && System.currentTimeMillis() < 
deadline) {
+   Thread.sleep(1);
+   }
+   assertTrue(buffer.isRecycled());
--- End diff --

No, it's not recycled in `partition.releaseMemory()` directly (or at least 
should not! - which is fixed now). The buffer will be recycled by the 
asynchronous writer thread once its content has been written to disk, i.e. 
after the `partition.releaseMemory()` call - I included such a check right 
before that call but actually, this is of limited use.


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-21 Thread NicoK
Github user NicoK commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r152342531
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
 ---
@@ -108,11 +108,7 @@ int releaseMemory() throws IOException {
for (int i = 0; i < numBuffers; i++) {
Buffer buffer = buffers.remove();
spilledBytes += buffer.getSize();
-   try {
-   spillWriter.writeBlock(buffer);
-   } finally {
-   buffer.recycle();
-   }
+   spillWriter.writeBlock(buffer);
--- End diff --

Actually, if I see this correctly, here the original code is wrong since it 
is already recycling a buffer which was added to an asynchronous file write 
operation. This would lead to data corruption if the buffer is re-used in the 
meanwhile, wouldn't it?!


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-21 Thread NicoK
Github user NicoK commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r152342364
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java
 ---
@@ -31,9 +31,26 @@ protected AsynchronousBufferFileWriter(ID channelID, 
RequestQueue
super(channelID, requestQueue, CALLBACK, true);
}
 
+   /**
+* Writes the given block asynchronously.
+*
+* @param buffer
+*  the buffer to be written (will be recycled when done)
--- End diff --

good catch, but actually, `SpillableSubpartition` doesn't do any recycling 
itself: in its `finish()` method, it relies on the buffer being on-heap and 
then garbage-collected, for the `add()` function, it relies on the caller, i.e. 
`ResultPartition#add()` (which I also forgot to adapt).


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-02 Thread pnowojski
Github user pnowojski commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r148534322
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
--- End diff --

nit: I'm evil I know, but could you replace mock `listener` with `new 
AwaitableBufferAvailablityListener()` if I please very nicely?


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-02 Thread pnowojski
Github user pnowojski commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r148531290
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousBufferFileWriter.java
 ---
@@ -31,9 +31,26 @@ protected AsynchronousBufferFileWriter(ID channelID, 
RequestQueue
super(channelID, requestQueue, CALLBACK, true);
}
 
+   /**
+* Writes the given block asynchronously.
+*
+* @param buffer
+*  the buffer to be written (will be recycled when done)
--- End diff --

Shouldn't this contract be in `BlockChannelWriterWithCallback`? Now it's 
kind of strange that recycle is implementation dependent. Especially that 
`SpillableSubpartition` is using `BufferFileWriter` and still depends on 
recycling. 


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-02 Thread pnowojski
Github user pnowojski commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r148531416
  
--- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionView.java
 ---
@@ -108,11 +108,7 @@ int releaseMemory() throws IOException {
for (int i = 0; i < numBuffers; i++) {
Buffer buffer = buffers.remove();
spilledBytes += buffer.getSize();
-   try {
-   spillWriter.writeBlock(buffer);
-   } finally {
-   buffer.recycle();
-   }
+   spillWriter.writeBlock(buffer);
--- End diff --

Here is the place where you depend on recycling which is not guaranteed by 
`spillWriter`'s interface.


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-02 Thread pnowojski
Github user pnowojski commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r148537550
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -231,31 +255,48 @@ public void 
testConsumeSpillablePartitionSpilledDuringConsume() throws Exception
 
// Initial notification
assertEquals(1, listener.getNumNotifiedBuffers());
+   assertFalse(buffer.isRecycled());
 
Buffer read = reader.getNextBuffer();
-   assertNotNull(read);
+   assertSame(buffer, read);
read.recycle();
assertEquals(2, listener.getNumNotifiedBuffers());
+   assertFalse(buffer.isRecycled());
 
// Spill now
assertEquals(2, partition.releaseMemory());
+   assertFalse(buffer.isRecycled()); // still one in the reader!
 
listener.awaitNotifications(4, 30_000);
assertEquals(4, listener.getNumNotifiedBuffers());
 
read = reader.getNextBuffer();
-   assertNotNull(read);
+   assertSame(buffer, read);
read.recycle();
+   // now the buffer may be freed, depending on the timing of the 
write operation
+   // -> let's do this check at the end of the test (to save some 
time)
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
// End of partition
read = reader.getNextBuffer();
assertNotNull(read);
assertEquals(EndOfPartitionEvent.class, 
EventSerializer.fromBuffer(read, 
ClassLoader.getSystemClassLoader()).getClass());
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
+
+   // finally check that the buffer has been freed after a 
successful (or failed) write
--- End diff --

ditto failed write case and does this test have anything to do with fixed 
bug?


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-02 Thread pnowojski
Github user pnowojski commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r148532775
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
Buffer read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
--- End diff --

Are those `read.isRecycled()` checks somehow related to this change/fix? Or 
just additional side hotfix? If the latter one, I would prefer to have them in 
separate commit because they are confusing me.


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-02 Thread pnowojski
Github user pnowojski commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r148533217
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
Buffer read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
// End of partition
read = reader.getNextBuffer();
assertNotNull(read);
assertEquals(EndOfPartitionEvent.class, 
EventSerializer.fromBuffer(read, 
ClassLoader.getSystemClassLoader()).getClass());
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
+
+   // finally check that the buffer has been freed after a 
successful (or failed) write
--- End diff --

Is this testing for failed write anywhere? Could this test catch previous 
error in double recycling? 


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-11-02 Thread pnowojski
Github user pnowojski commented on a diff in the pull request:

https://github.com/apache/flink/pull/4581#discussion_r148532507
  
--- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/SpillableSubpartitionTest.java
 ---
@@ -192,21 +198,39 @@ public void testConsumeSpilledPartition() throws 
Exception {
 
Buffer read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
read = reader.getNextBuffer();
assertNotNull(read);
+   assertNotSame(buffer, read);
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
 
// End of partition
read = reader.getNextBuffer();
assertNotNull(read);
assertEquals(EndOfPartitionEvent.class, 
EventSerializer.fromBuffer(read, 
ClassLoader.getSystemClassLoader()).getClass());
+   assertFalse(read.isRecycled());
read.recycle();
+   assertTrue(read.isRecycled());
+
+   // finally check that the buffer has been freed after a 
successful (or failed) write
+   final long deadline = System.currentTimeMillis() + 30_000L; // 
30 secs
+   while (!buffer.isRecycled() && System.currentTimeMillis() < 
deadline) {
+   Thread.sleep(1);
+   }
+   assertTrue(buffer.isRecycled());
--- End diff --

Where is the last place in this test were you can add 
`assertFalse(buffer.isRecycled())`? I think placing it somewhere would help to 
understand what's going on and would be nice sanity check.

~~Isn't the `buffer` recycled in `partition.releaseMemory()` call far 
above?~~ yes it is ;)


---


[GitHub] flink pull request #4581: [FLINK-7499][io] also let AsynchronousBufferFileWr...

2017-08-24 Thread NicoK
GitHub user NicoK opened a pull request:

https://github.com/apache/flink/pull/4581

[FLINK-7499][io] also let AsynchronousBufferFileWriter#writeBlock() recycle 
the buffer in case of failures

## What is the purpose of the change

`SpillableSubpartitionView#releaseMemory()` recycled the given buffer 
twice: once asynchronously after the write operation initiated via 
`AsynchronousBufferFileWriter#writeBlock()`and once in 
SpillableSubpartitionView#releaseMemory() after adding the write operation to 
the queue. Additionally, other uses of 
`AsynchronousBufferFileWriter#writeBlock()` did not cleanup in an error case 
which was also not done by `AsynchronousBufferFileWriter#writeBlock()` itself.

This PR changes the behaviour of 
`AsynchronousBufferFileWriter#writeBlock()` to always take care of releasing 
the buffer, even adding the (asynchronous) write operation failed.

## Brief change log

- let `AsynchronousBufferFileWriter#writeBlock()` take full recycling 
responsibility of the given buffer even in case of failures
- remove the additional `recycle` call in 
`SpillableSubpartitionView#releaseMemory()`
- adapt `SpillableSubpartitionTest` to find the duplicate `recycle()` calls

## Verifying this change

This change added further checks to `SpillableSubpartitionTest` to verify 
the intended behaviour

## Does this pull request potentially affect one of the following parts:

  - Dependencies (does it add or upgrade a dependency): (no)
  - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
  - The serializers: (no)
  - The runtime per-record code paths (performance sensitive): (yes)
  - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (no)

## Documentation

  - Does this pull request introduce a new feature? (no)
  - If yes, how is the feature documented? (not applicable)


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/NicoK/flink flink-7499

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/flink/pull/4581.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #4581


commit a986ebac2da4ac4ad00717e834fdc33f9fe9eb3a
Author: Nico Kruber 
Date:   2017-08-24T10:17:08Z

[FLINK-7499][io] also let AsynchronousBufferFileWriter#writeBlock() recycle 
the buffer in case of failures

This fixes a double-recycle in SpillableSubpartitionView and also makes sure
that even if adding the (asynchronous) write operation fails, the buffer is
properly freed in code that did not perform this cleanup. It avoids code
duplication of this cleanup and it is also more consistent to take over
responsibility of the given buffer even if an exception is thrown.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---