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 2015/09/08 20:59:02 UTC

[12/15] flink git commit: [FLINK-1320] [core] Add an off-heap variant of the managed memory

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java
new file mode 100644
index 0000000..7b8d996
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentUndersizedTest.java
@@ -0,0 +1,1367 @@
+/*
+ * 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.core.memory;
+
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class MemorySegmentUndersizedTest {
+	
+	@Test
+	public void testZeroSizeHeapSegment() {
+		try {
+			MemorySegment segment = new HeapMemorySegment(new byte[0]);
+
+			testZeroSizeBuffer(segment);
+			testSegmentWithSizeLargerZero(segment);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testZeroSizeHeapHybridSegment() {
+		try {
+			MemorySegment segment = new HybridMemorySegment(new byte[0]);
+
+			testZeroSizeBuffer(segment);
+			testSegmentWithSizeLargerZero(segment);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testZeroSizeOffHeapHybridSegment() {
+		try {
+			MemorySegment segment = new HybridMemorySegment(ByteBuffer.allocateDirect(0));
+
+			testZeroSizeBuffer(segment);
+			testSegmentWithSizeLargerZero(segment);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSizeOneHeapSegment() {
+		try {
+			testSegmentWithSizeLargerZero(new HeapMemorySegment(new byte[1]));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSizeOneHeapHybridSegment() {
+		try {
+			testSegmentWithSizeLargerZero(new HybridMemorySegment(new byte[1]));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSizeOneOffHeapHybridSegment() {
+		try {
+			testSegmentWithSizeLargerZero(new HybridMemorySegment(ByteBuffer.allocateDirect(1)));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+
+	private static void testZeroSizeBuffer(MemorySegment segment) {
+		// ------ bytes ------
+
+		try {
+			segment.put(0, (byte) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ booleans ------
+
+		try {
+			segment.putBoolean(0, true);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getBoolean(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+	}
+	
+	private static void testSegmentWithSizeLargerZero(MemorySegment segment) {
+
+		// ------ bytes ------
+
+		try {
+			segment.put(1, (byte) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(-1, (byte) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(8, (byte) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(-8, (byte) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.put(Integer.MAX_VALUE, (byte) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(Integer.MIN_VALUE, (byte) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.get(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.get(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.get(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ booleans ------
+
+		try {
+			segment.putBoolean(1, true);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putBoolean(-1, true);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putBoolean(8, true);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putBoolean(-8, true);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.putBoolean(Integer.MAX_VALUE, true);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putBoolean(Integer.MIN_VALUE, true);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.getBoolean(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getBoolean(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getBoolean(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getBoolean(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getBoolean(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getBoolean(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ chars ------
+
+		try {
+			segment.putChar(0, 'a');
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putChar(1, 'a');
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putChar(-1, 'a');
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putChar(8, 'a');
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putChar(-8, 'a');
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putChar(Integer.MAX_VALUE, 'a');
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putChar(Integer.MIN_VALUE, 'a');
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getChar(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.getChar(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getChar(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getChar(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getChar(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getChar(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getChar(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ shorts ------
+
+		try {
+			segment.putShort(0, (short) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putShort(1, (short) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putShort(-1, (short) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putShort(8, (short) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putShort(-8, (short) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putShort(Integer.MAX_VALUE, (short) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putShort(Integer.MIN_VALUE, (short) 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getShort(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.getShort(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getShort(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getShort(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getShort(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getShort(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getShort(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ ints ------
+
+		try {
+			segment.putInt(0, 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putInt(1, 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putInt(-1, 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putInt(8, 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putInt(-8, 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putInt(Integer.MAX_VALUE, 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putInt(Integer.MIN_VALUE, 0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getInt(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		try {
+			segment.getInt(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getInt(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getInt(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getInt(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getInt(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getInt(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ longs ------
+
+		try {
+			segment.putLong(0, 0L);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putLong(1, 0L);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putLong(-1, 0L);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putLong(8, 0L);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putLong(-8, 0L);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putLong(Integer.MAX_VALUE, 0L);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putLong(Integer.MIN_VALUE, 0L);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getLong(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		try {
+			segment.getLong(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getLong(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getLong(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getLong(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getLong(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getLong(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ floats ------
+
+		try {
+			segment.putFloat(0, 0.0f);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putFloat(1, 0.0f);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putFloat(-1, 0.0f);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putFloat(8, 0.0f);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putFloat(-8, 0.0f);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putFloat(Integer.MAX_VALUE, 0.0f);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putFloat(Integer.MIN_VALUE, 0.0f);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getFloat(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		try {
+			segment.getFloat(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getFloat(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getFloat(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getFloat(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getFloat(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getFloat(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ doubles ------
+
+		try {
+			segment.putDouble(0, 0.0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putDouble(1, 0.0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putDouble(-1, 0.0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putDouble(8, 0.0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putDouble(Integer.MAX_VALUE, 0.0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.putDouble(Integer.MIN_VALUE, 0.0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getDouble(0);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		try {
+			segment.getDouble(1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getDouble(-1);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getDouble(8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getDouble(-8);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.getDouble(Integer.MAX_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.getDouble(Integer.MIN_VALUE);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+
+		// ------ byte[] ------
+
+		try {
+			segment.put(0, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(1, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(-1, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(8, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(-8, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(Integer.MAX_VALUE, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(Integer.MIN_VALUE, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(0, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.get(1, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(-1, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(8, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(-8, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(Integer.MAX_VALUE, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(Integer.MIN_VALUE, new byte[7]);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ ByteBuffer ------
+
+		final ByteBuffer buf = ByteBuffer.allocate(7);
+		final int numBytes = 3; 
+		
+		try {
+			segment.put(0, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(1, buf, numBytes);
+					fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(-1, buf, numBytes);
+					fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(8, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(-8, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		
+		try {
+			segment.put(Integer.MAX_VALUE, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(Integer.MIN_VALUE, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(0, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		try {
+			segment.get(1, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(-1, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(8, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(-8, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(Integer.MAX_VALUE, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(Integer.MIN_VALUE, buf, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		// ------ DataInput / DataOutput ------
+
+		final DataInput dataInput = new DataInputStream(new ByteArrayInputStream(new byte[20]));
+		final DataOutput dataOutput = new DataOutputStream(new ByteArrayOutputStream());
+		
+		try {
+			segment.put(dataInput, 0, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(dataInput, 1, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(dataInput, -1, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(dataInput, 8, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(dataInput, -8, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(dataInput, Integer.MAX_VALUE, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.put(dataInput, Integer.MIN_VALUE, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(dataOutput, 0, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+		try {
+			segment.get(dataOutput, 1, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(dataOutput, -1, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(dataOutput, 8, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(dataOutput, -8, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(dataOutput, Integer.MAX_VALUE, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+
+		try {
+			segment.get(dataOutput, Integer.MIN_VALUE, numBytes);
+			fail("IndexOutOfBoundsException expected");
+		}
+		catch (Exception e) {
+			assertTrue(e instanceof IndexOutOfBoundsException);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java
new file mode 100644
index 0000000..ddbdf72
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/OperationsOnFreedSegmentTest.java
@@ -0,0 +1,1195 @@
+/*
+ * 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.core.memory;
+
+import org.junit.Test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.*;
+
+public class OperationsOnFreedSegmentTest {
+
+	private static final int PAGE_SIZE = (int) ((Math.random() * 10000) + 1000);
+	
+	@Test
+	public void testSingleSegmentOperationsHeapSegment() {
+		try {
+			testOpsOnFreedSegment(new HeapMemorySegment(new byte[PAGE_SIZE]));
+			testOpsOnFreedSegment(new HybridMemorySegment(new byte[PAGE_SIZE]));
+			testOpsOnFreedSegment(new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE)));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testCompare() {
+		try {
+			MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+
+			MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+			freedHeap.free();
+			freedHybridHeap.free();
+			freedHybridOffHeap.free();
+			
+			MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap };
+			MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap };
+			
+			// alive with free
+			for (MemorySegment seg1 : alive) {
+				for (MemorySegment seg2 : free) {
+					testCompare(seg1, seg2);
+				}
+			}
+
+			// free with alive
+			for (MemorySegment seg1 : free) {
+				for (MemorySegment seg2 : alive) {
+					testCompare(seg1, seg2);
+				}
+			}
+
+			// free with free
+			for (MemorySegment seg1 : free) {
+				for (MemorySegment seg2 : free) {
+					testCompare(seg1, seg2);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testCopyTo() {
+		try {
+			MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+
+			MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+			freedHeap.free();
+			freedHybridHeap.free();
+			freedHybridOffHeap.free();
+
+			MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap };
+			MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap };
+
+			// alive with free
+			for (MemorySegment seg1 : alive) {
+				for (MemorySegment seg2 : free) {
+					testCopy(seg1, seg2);
+				}
+			}
+
+			// free with alive
+			for (MemorySegment seg1 : free) {
+				for (MemorySegment seg2 : alive) {
+					testCopy(seg1, seg2);
+				}
+			}
+
+			// free with free
+			for (MemorySegment seg1 : free) {
+				for (MemorySegment seg2 : free) {
+					testCopy(seg1, seg2);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSwap() {
+		try {
+			MemorySegment aliveHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment aliveHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment aliveHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+
+			MemorySegment freedHeap = new HeapMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment freedHybridHeap = new HybridMemorySegment(new byte[PAGE_SIZE]);
+			MemorySegment freedHybridOffHeap = new HybridMemorySegment(ByteBuffer.allocateDirect(PAGE_SIZE));
+			freedHeap.free();
+			freedHybridHeap.free();
+			freedHybridOffHeap.free();
+
+			MemorySegment[] alive = { aliveHeap, aliveHybridHeap, aliveHybridOffHeap };
+			MemorySegment[] free = { freedHeap, freedHybridHeap, freedHybridOffHeap };
+
+			// alive with free
+			for (MemorySegment seg1 : alive) {
+				for (MemorySegment seg2 : free) {
+					testSwap(seg1, seg2);
+				}
+			}
+
+			// free with alive
+			for (MemorySegment seg1 : free) {
+				for (MemorySegment seg2 : alive) {
+					testSwap(seg1, seg2);
+				}
+			}
+
+			// free with free
+			for (MemorySegment seg1 : free) {
+				for (MemorySegment seg2 : free) {
+					testSwap(seg1, seg2);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private void testOpsOnFreedSegment(MemorySegment segment) throws Exception {
+		segment.free();
+		assertTrue(segment.isFreed());
+		
+		// --------- bytes ----------- 
+		
+		try {
+			segment.get(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.get(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+		
+		try {
+			segment.get(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(0, (byte) 0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(-1, (byte) 0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(1, (byte) 0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(segment.size(), (byte) 0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(-segment.size(), (byte) 0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(Integer.MAX_VALUE, (byte) 0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(Integer.MIN_VALUE, (byte) 0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		// --------- booleans ----------- 
+
+		try {
+			segment.getBoolean(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.getBoolean(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getBoolean(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.getBoolean(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.getBoolean(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getBoolean(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.getBoolean(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.putBoolean(0, true);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.putBoolean(-1, true);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putBoolean(1, true);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.putBoolean(segment.size(), true);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.putBoolean(-segment.size(), true);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putBoolean(Integer.MAX_VALUE, true);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.putBoolean(Integer.MIN_VALUE, true);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+		
+		// --------- char ----------- 
+
+		try {
+			segment.getChar(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getChar(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getChar(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getChar(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getChar(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getChar(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getChar(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putChar(0, 'a');
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putChar(-1, 'a');
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putChar(1, 'a');
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putChar(segment.size(), 'a');
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putChar(-segment.size(), 'a');
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putChar(Integer.MAX_VALUE, 'a');
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putChar(Integer.MIN_VALUE, 'a');
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		// --------- short ----------- 
+
+		try {
+			segment.getShort(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getShort(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getShort(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getShort(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getShort(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getShort(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getShort(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putShort(0, (short) 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putShort(-1, (short) 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putShort(1, (short) 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putShort(segment.size(), (short) 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putShort(-segment.size(), (short) 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putShort(Integer.MAX_VALUE, (short) 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putShort(Integer.MIN_VALUE, (short) 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		// --------- integer ----------- 
+
+		try {
+			segment.getInt(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getInt(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getInt(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getInt(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getInt(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getInt(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getInt(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putInt(0, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putInt(-1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putInt(1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putInt(segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putInt(-segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putInt(Integer.MAX_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putInt(Integer.MIN_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		// --------- longs ----------- 
+
+		try {
+			segment.getLong(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getLong(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getLong(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getLong(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getLong(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getLong(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getLong(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putLong(0, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putLong(-1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putLong(1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putLong(segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putLong(-segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putLong(Integer.MAX_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putLong(Integer.MIN_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		// --------- floats ----------- 
+
+		try {
+			segment.getFloat(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getFloat(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getFloat(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getFloat(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getFloat(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getFloat(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getFloat(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putFloat(0, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putFloat(-1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putFloat(1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putFloat(segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putFloat(-segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putFloat(Integer.MAX_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putFloat(Integer.MIN_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		// --------- doubles ----------- 
+
+		try {
+			segment.getDouble(0);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getDouble(-1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getDouble(1);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getDouble(segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getDouble(-segment.size());
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.getDouble(Integer.MAX_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.getDouble(Integer.MIN_VALUE);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putDouble(0, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putDouble(-1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putDouble(1, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putDouble(segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putDouble(-segment.size(), 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.putDouble(Integer.MAX_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		try {
+			segment.putDouble(Integer.MIN_VALUE, 42);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException ignored) {}
+
+		// --------- byte[] ----------- 
+
+		final byte[] array = new byte[55];
+		
+		try {
+			segment.get(0, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(-1, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.get(1, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(segment.size(), array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(-segment.size(), array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.get(Integer.MAX_VALUE, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(Integer.MIN_VALUE, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(0, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(-1, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(1, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(segment.size(), array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(-segment.size(), array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(Integer.MAX_VALUE, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(Integer.MIN_VALUE, array, 3, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		// --------- ByteBuffer ----------- 
+
+		for (ByteBuffer bbuf : new ByteBuffer[] {
+				ByteBuffer.allocate(55),
+				ByteBuffer.allocateDirect(55) } )
+		{
+			try {
+				segment.get(0, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException ignored) {}
+	
+			try {
+				segment.get(-1, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.get(1, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException ignored) {}
+	
+			try {
+				segment.get(segment.size(), bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException ignored) {}
+	
+			try {
+				segment.get(-segment.size(), bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.get(Integer.MAX_VALUE, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.get(Integer.MIN_VALUE, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.put(0, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.put(-1, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.put(1, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException ignored) {}
+	
+			try {
+				segment.put(segment.size(), bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException ignored) {}
+	
+			try {
+				segment.put(-segment.size(), bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.put(Integer.MAX_VALUE,bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	
+			try {
+				segment.put(Integer.MIN_VALUE, bbuf, 17);
+				fail("Should fail with an exception");
+			}
+			catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+		}
+
+		// --------- Data Input / Output ----------- 
+
+		final DataInput din = new DataInputStream(new ByteArrayInputStream(new byte[100]));
+		final DataOutput dout = new DataOutputStream(new ByteArrayOutputStream());
+
+		try {
+			segment.get(dout, 0, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(dout, -1, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.get(dout, 1, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(dout, segment.size(), 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.get(dout, -segment.size(), 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.get(dout, Integer.MAX_VALUE, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.get(dout, Integer.MIN_VALUE, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(din, 0, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(din, -1, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(din, 1, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(din, segment.size(), 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException ignored) {}
+
+		try {
+			segment.put(din, -segment.size(), 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(din, Integer.MAX_VALUE, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+
+		try {
+			segment.put(din, Integer.MIN_VALUE, 17);
+			fail("Should fail with an exception");
+		}
+		catch (IllegalStateException | NullPointerException | IndexOutOfBoundsException ignored) {}
+	}
+	
+	
+	private void testCompare(MemorySegment seg1, MemorySegment seg2) {
+		int[] offsetsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+		int[] lengthsToTest = { 1, seg1.size(), Integer.MAX_VALUE };
+		
+		for (int off1 : offsetsToTest) {
+			for (int off2 : offsetsToTest) {
+				for (int len : lengthsToTest) {
+					try {
+						seg1.compare(seg2, off1, off2, len);
+						fail("Should fail with an exception");
+					}
+					catch (IllegalStateException | IndexOutOfBoundsException | NullPointerException ignored) {}
+				}
+			}
+		}
+	}
+
+	private void testCopy(MemorySegment seg1, MemorySegment seg2) {
+		int[] offsetsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+		int[] lengthsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+		for (int off1 : offsetsToTest) {
+			for (int off2 : offsetsToTest) {
+				for (int len : lengthsToTest) {
+					try {
+						seg1.copyTo(off1, seg2, off2, len);
+						fail("Should fail with an exception");
+					}
+					catch (IllegalStateException | IndexOutOfBoundsException | NullPointerException ignored) {}
+				}
+			}
+		}
+	}
+
+	private void testSwap(MemorySegment seg1, MemorySegment seg2) {
+		int[] offsetsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+		int[] lengthsToTest = { 0, 1, -1, seg1.size(), -seg1.size(), Integer.MAX_VALUE, Integer.MIN_VALUE };
+		byte[] swapBuffer = new byte[seg1.size()];
+		
+		for (int off1 : offsetsToTest) {
+			for (int off2 : offsetsToTest) {
+				for (int len : lengthsToTest) {
+					try {
+						seg1.swapBytes(swapBuffer, seg2, off1, off2, len);
+						fail("Should fail with an exception");
+					}
+					catch (IllegalStateException | IndexOutOfBoundsException | NullPointerException ignored) {}
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java
new file mode 100644
index 0000000..df3f8be
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java
@@ -0,0 +1,360 @@
+/*
+ * 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.core.memory.benchmarks;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class CoreMemorySegmentOutView implements DataOutputView {
+
+	private MemorySegment currentSegment;	// the current memory segment to write to
+
+	private int positionInSegment;					// the offset in the current segment
+	
+	private final int segmentSize;				// the size of the memory segments
+
+	private final  List<MemorySegment> memorySource;
+	
+	private final List<MemorySegment> fullSegments;
+	
+
+	private byte[] utfBuffer;		// the reusable array for UTF encodings
+
+
+	public CoreMemorySegmentOutView(List<MemorySegment> emptySegments,
+									List<MemorySegment> fullSegmentTarget, int segmentSize) {
+		this.segmentSize = segmentSize;
+		this.currentSegment = emptySegments.remove(emptySegments.size() - 1);
+
+		this.memorySource = emptySegments;
+		this.fullSegments = fullSegmentTarget;
+		this.fullSegments.add(getCurrentSegment());
+	}
+
+
+	public void reset() {
+		if (this.fullSegments.size() != 0) {
+			throw new IllegalStateException("The target list still contains memory segments.");
+		}
+
+		clear();
+		try {
+			advance();
+		}
+		catch (IOException ioex) {
+			throw new RuntimeException("Error getting first segment for record collector.", ioex);
+		}
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                                  Page Management
+	// --------------------------------------------------------------------------------------------
+
+	public MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws EOFException {
+		int size = this.memorySource.size();
+		if (size > 0) {
+			final MemorySegment next = this.memorySource.remove(size - 1);
+			this.fullSegments.add(next);
+			return next;
+		} else {
+			throw new EOFException();
+		}
+	}
+	
+	public MemorySegment getCurrentSegment() {
+		return this.currentSegment;
+	}
+
+	public int getCurrentPositionInSegment() {
+		return this.positionInSegment;
+	}
+	
+	public int getSegmentSize() {
+		return this.segmentSize;
+	}
+	
+	protected void advance() throws IOException {
+		this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
+		this.positionInSegment = 0;
+	}
+	
+	protected void seekOutput(MemorySegment seg, int position) {
+		this.currentSegment = seg;
+		this.positionInSegment = position;
+	}
+
+	protected void clear() {
+		this.currentSegment = null;
+		this.positionInSegment = 0;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                               Data Output Specific methods
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public void write(int b) throws IOException {
+		writeByte(b);
+	}
+
+	@Override
+	public void write(byte[] b) throws IOException {
+		write(b, 0, b.length);
+	}
+
+	@Override
+	public void write(byte[] b, int off, int len) throws IOException {
+		int remaining = this.segmentSize - this.positionInSegment;
+		if (remaining >= len) {
+			this.currentSegment.put(this.positionInSegment, b, off, len);
+			this.positionInSegment += len;
+		}
+		else {
+			if (remaining == 0) {
+				advance();
+				remaining = this.segmentSize - this.positionInSegment;
+			}
+			while (true) {
+				int toPut = Math.min(remaining, len);
+				this.currentSegment.put(this.positionInSegment, b, off, toPut);
+				off += toPut;
+				len -= toPut;
+
+				if (len > 0) {
+					this.positionInSegment = this.segmentSize;
+					advance();
+					remaining = this.segmentSize - this.positionInSegment;
+				}
+				else {
+					this.positionInSegment += toPut;
+					break;
+				}
+			}
+		}
+	}
+
+	@Override
+	public void writeBoolean(boolean v) throws IOException {
+		writeByte(v ? 1 : 0);
+	}
+
+	@Override
+	public void writeByte(int v) throws IOException {
+		if (this.positionInSegment < this.segmentSize) {
+			this.currentSegment.put(this.positionInSegment++, (byte) v);
+		}
+		else {
+			advance();
+			writeByte(v);
+		}
+	}
+
+	@Override
+	public void writeShort(int v) throws IOException {
+		if (this.positionInSegment < this.segmentSize - 1) {
+			this.currentSegment.putShortBigEndian(this.positionInSegment, (short) v);
+			this.positionInSegment += 2;
+		}
+		else if (this.positionInSegment == this.segmentSize) {
+			advance();
+			writeShort(v);
+		}
+		else {
+			writeByte(v >> 8);
+			writeByte(v);
+		}
+	}
+
+	@Override
+	public void writeChar(int v) throws IOException {
+		if (this.positionInSegment < this.segmentSize - 1) {
+			this.currentSegment.putCharBigEndian(this.positionInSegment, (char) v);
+			this.positionInSegment += 2;
+		}
+		else if (this.positionInSegment == this.segmentSize) {
+			advance();
+			writeChar(v);
+		}
+		else {
+			writeByte(v >> 8);
+			writeByte(v);
+		}
+	}
+
+	@Override
+	public void writeInt(int v) throws IOException {
+		if (this.positionInSegment < this.segmentSize - 3) {
+			this.currentSegment.putIntBigEndian(this.positionInSegment, v);
+			this.positionInSegment += 4;
+		}
+		else if (this.positionInSegment == this.segmentSize) {
+			advance();
+			writeInt(v);
+		}
+		else {
+			writeByte(v >> 24);
+			writeByte(v >> 16);
+			writeByte(v >>  8);
+			writeByte(v);
+		}
+	}
+
+	@Override
+	public void writeLong(long v) throws IOException {
+		if (this.positionInSegment < this.segmentSize - 7) {
+			this.currentSegment.putLongBigEndian(this.positionInSegment, v);
+			this.positionInSegment += 8;
+		}
+		else if (this.positionInSegment == this.segmentSize) {
+			advance();
+			writeLong(v);
+		}
+		else {
+			writeByte((int) (v >> 56));
+			writeByte((int) (v >> 48));
+			writeByte((int) (v >> 40));
+			writeByte((int) (v >> 32));
+			writeByte((int) (v >> 24));
+			writeByte((int) (v >> 16));
+			writeByte((int) (v >>  8));
+			writeByte((int) v);
+		}
+	}
+
+	@Override
+	public void writeFloat(float v) throws IOException {
+		writeInt(Float.floatToRawIntBits(v));
+	}
+
+	@Override
+	public void writeDouble(double v) throws IOException {
+		writeLong(Double.doubleToRawLongBits(v));
+	}
+
+	@Override
+	public void writeBytes(String s) throws IOException {
+		for (int i = 0; i < s.length(); i++) {
+			writeByte(s.charAt(i));
+		}
+	}
+
+	@Override
+	public void writeChars(String s) throws IOException {
+		for (int i = 0; i < s.length(); i++) {
+			writeChar(s.charAt(i));
+		}
+	}
+
+	@Override
+	public void writeUTF(String str) throws IOException {
+		int strlen = str.length();
+		int utflen = 0;
+		int c, count = 0;
+
+		/* use charAt instead of copying String to char array */
+		for (int i = 0; i < strlen; i++) {
+			c = str.charAt(i);
+			if ((c >= 0x0001) && (c <= 0x007F)) {
+				utflen++;
+			} else if (c > 0x07FF) {
+				utflen += 3;
+			} else {
+				utflen += 2;
+			}
+		}
+
+		if (utflen > 65535) {
+			throw new UTFDataFormatException("encoded string too long: " + utflen + " memory");
+		}
+
+		if (this.utfBuffer == null || this.utfBuffer.length < utflen + 2) {
+			this.utfBuffer = new byte[utflen + 2];
+		}
+		final byte[] bytearr = this.utfBuffer;
+
+		bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF);
+		bytearr[count++] = (byte) (utflen & 0xFF);
+
+		int i = 0;
+		for (i = 0; i < strlen; i++) {
+			c = str.charAt(i);
+			if (!((c >= 0x0001) && (c <= 0x007F))) {
+				break;
+			}
+			bytearr[count++] = (byte) c;
+		}
+
+		for (; i < strlen; i++) {
+			c = str.charAt(i);
+			if ((c >= 0x0001) && (c <= 0x007F)) {
+				bytearr[count++] = (byte) c;
+
+			} else if (c > 0x07FF) {
+				bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F));
+				bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F));
+				bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+			} else {
+				bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
+				bytearr[count++] = (byte) (0x80 | (c & 0x3F));
+			}
+		}
+
+		write(bytearr, 0, utflen + 2);
+	}
+
+	@Override
+	public void skipBytesToWrite(int numBytes) throws IOException {
+		while (numBytes > 0) {
+			final int remaining = this.segmentSize - this.positionInSegment;
+			if (numBytes <= remaining) {
+				this.positionInSegment += numBytes;
+				return;
+			}
+			this.positionInSegment = this.segmentSize;
+			advance();
+			numBytes -= remaining;
+		}
+	}
+
+	@Override
+	public void write(DataInputView source, int numBytes) throws IOException {
+		while (numBytes > 0) {
+			final int remaining = this.segmentSize - this.positionInSegment;
+			if (numBytes <= remaining) {
+				this.currentSegment.put(source, this.positionInSegment, numBytes);
+				this.positionInSegment += numBytes;
+				return;
+			}
+
+			if (remaining > 0) {
+				this.currentSegment.put(source, this.positionInSegment, remaining);
+				this.positionInSegment = this.segmentSize;
+				numBytes -= remaining;
+			}
+
+			advance();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java
new file mode 100644
index 0000000..ed3c1f5
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java
@@ -0,0 +1,232 @@
+/*
+ * 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.core.memory.benchmarks;
+
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Random;
+
+public class LongSerializationSpeedBenchmark {
+	
+	public static void main(String[] args) throws Exception {
+		
+		final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
+		final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
+		final ByteBuffer largeOffHeap = ByteBuffer.allocateDirect(LARGE_SEGMENT_SIZE);
+
+		fillOnHeap(largeSegment, (byte) -1);
+		fillOffHeap(largeOffHeap, (byte) -1);
+		
+		final MemorySegment coreHeap = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null);
+		final MemorySegment coreHybridOnHeap = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null);
+		final MemorySegment coreHybridOffHeap = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(largeOffHeap, null);
+		final PureHeapMemorySegment pureHeap = new PureHeapMemorySegment(largeSegment);
+		final PureHybridMemorySegment pureHybridOnHeap = new PureHybridMemorySegment(largeSegment);
+		final PureHybridMemorySegment pureHybridOffHeap = new PureHybridMemorySegment(largeOffHeap);
+		
+		final LongSerializer ser = LongSerializer.INSTANCE;
+		
+		final long innerRounds = LARGE_SEGMENT_SIZE / 8;
+		final int outerRounds = 10;
+
+		{
+			System.out.println("testing core heap memory segment");
+
+			ArrayList<MemorySegment> memory = new ArrayList<>();
+			memory.add(coreHeap);
+			ArrayList<MemorySegment> target = new ArrayList<>();
+
+			CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+				for (long i = 0; i < innerRounds; i++) {
+					ser.serialize(i, output);
+				}
+
+				target.clear();
+				memory.add(coreHeap);
+				output.reset();
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Core heap memory segment took %,d msecs", (stop - start) / 1000000));
+		}
+
+		{
+			System.out.println("testing core hybrid on heap memory segment");
+
+			ArrayList<MemorySegment> memory = new ArrayList<>();
+			memory.add(coreHybridOnHeap);
+			ArrayList<MemorySegment> target = new ArrayList<>();
+
+			CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+				for (long i = 0; i < innerRounds; i++) {
+					ser.serialize(i, output);
+				}
+
+				target.clear();
+				memory.add(coreHybridOnHeap);
+				output.reset();
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Core hybrid on heap memory segment took %,d msecs", (stop - start) / 1000000));
+		}
+
+		{
+			System.out.println("testing core hybrid off heap memory segment");
+
+			ArrayList<MemorySegment> memory = new ArrayList<>();
+			memory.add(coreHybridOffHeap);
+			ArrayList<MemorySegment> target = new ArrayList<>();
+
+			CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+				for (long i = 0; i < innerRounds; i++) {
+					ser.serialize(i, output);
+				}
+
+				target.clear();
+				memory.add(coreHybridOffHeap);
+				output.reset();
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Core hybrid off heap memory segment took %,d msecs", (stop - start) / 1000000));
+		}
+		
+		{
+			System.out.println("testing pure heap memory segment");
+
+			ArrayList<PureHeapMemorySegment> memory = new ArrayList<>();
+			memory.add(pureHeap);
+			ArrayList<PureHeapMemorySegment> target = new ArrayList<>();
+
+			PureHeapMemorySegmentOutView output = new PureHeapMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+			
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+				for (long i = 0; i < innerRounds; i++) {
+					ser.serialize(i, output);
+				}
+
+				target.clear();
+				memory.add(pureHeap);
+				output.reset();
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Pure heap memory segment took %,d msecs", (stop - start) / 1000000));
+		}
+
+		{
+			System.out.println("testing pure hybrid memory segment on heap");
+
+			ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+			memory.add(pureHybridOnHeap);
+			ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+
+			PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+				for (long i = 0; i < innerRounds; i++) {
+					ser.serialize(i, output);
+				}
+
+				target.clear();
+				memory.add(pureHybridOnHeap);
+				output.reset();
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Pure hybrid on heap memory segment took %,d msecs", (stop - start) / 1000000));
+		}
+
+		{
+			System.out.println("testing pure hybrid memory segment off heap");
+
+			ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+			memory.add(pureHybridOffHeap);
+			ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+
+			PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+				for (long i = 0; i < innerRounds; i++) {
+					ser.serialize(i, output);
+				}
+
+				target.clear();
+				memory.add(pureHybridOffHeap);
+				output.reset();
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Pure hybrid off heap memory segment took %,d msecs", (stop - start) / 1000000));
+		}
+	}
+	
+	private static String[] generateRandomStrings(long seed, int num, int maxLen, int minLen, boolean asciiOnly) {
+		Random rnd = new Random(seed);
+		String[] array = new String[num];
+		StringBuilder bld = new StringBuilder(maxLen);
+		
+		int minCharValue = 40;
+		int charRange = asciiOnly ? 60 : 30000;
+		
+		for (int i = 0; i < num; i++) {
+			bld.setLength(0);
+			int len = rnd.nextInt(maxLen - minLen) + minLen;
+			
+			for (int k = 0; k < len; k++) {
+				bld.append((char) (rnd.nextInt(charRange) + minCharValue));
+			}
+			
+			array[i] = bld.toString();
+		}
+		
+		return array;
+	}
+
+	private static void fillOnHeap(byte[] buffer, byte data) {
+		for (int i = 0; i < buffer.length; i++) {
+			buffer[i] = data;
+		}
+	}
+
+	private static void fillOffHeap(ByteBuffer buffer, byte data) {
+		final int len = buffer.capacity();
+		for (int i = 0; i < len; i++) {
+			buffer.put(i, data);
+		}
+	}
+}