You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2014/12/15 17:57:38 UTC

incubator-flink git commit: [FLINK-1327] [core] Fix wrapping of MemorySegments in ByteBuffers

Repository: incubator-flink
Updated Branches:
  refs/heads/master c8fa1cd06 -> 283c398e4


[FLINK-1327] [core] Fix wrapping of MemorySegments in ByteBuffers


Project: http://git-wip-us.apache.org/repos/asf/incubator-flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-flink/commit/283c398e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-flink/tree/283c398e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-flink/diff/283c398e

Branch: refs/heads/master
Commit: 283c398e43239bd62afec90875894eb2a4e5b110
Parents: c8fa1cd
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Dec 15 16:46:07 2014 +0100
Committer: Stephan Ewen <se...@apache.org>
Committed: Mon Dec 15 16:46:07 2014 +0100

----------------------------------------------------------------------
 .../apache/flink/core/memory/MemorySegment.java |  2 +-
 .../flink/runtime/memory/MemorySegmentTest.java | 28 +++++++++++++++++++-
 2 files changed, 28 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/283c398e/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
index 5b6b175..9b4bae8 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegment.java
@@ -144,8 +144,8 @@ public class MemorySegment {
 			this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
 		}
 		else {
-			this.wrapper.position(offset);
 			this.wrapper.limit(offset + length);
+			this.wrapper.position(offset);
 		}
 		
 		return this.wrapper;

http://git-wip-us.apache.org/repos/asf/incubator-flink/blob/283c398e/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
index bfc5c49..cefd0c5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
@@ -24,10 +24,10 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.nio.ByteBuffer;
 import java.util.Random;
 
 import org.junit.Assert;
-
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
 import org.junit.After;
@@ -545,4 +545,30 @@ public class MemorySegmentTest {
 			}
 		}
 	}
+	
+	@Test
+	public void testByteBufferWrapping() {
+		try {
+			MemorySegment seg = new MemorySegment(new byte[1024]);
+			
+			ByteBuffer buf1 = seg.wrap(13, 47);
+			assertEquals(13, buf1.position());
+			assertEquals(60, buf1.limit());
+			assertEquals(47, buf1.remaining());
+			
+			ByteBuffer buf2 = seg.wrap(500, 267);
+			assertEquals(500, buf2.position());
+			assertEquals(767, buf2.limit());
+			assertEquals(267, buf2.remaining());
+			
+			ByteBuffer buf3 = seg.wrap(0, 1024);
+			assertEquals(0, buf3.position());
+			assertEquals(1024, buf3.limit());
+			assertEquals(1024, buf3.remaining());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+	}
 }