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:58:51 UTC

[01/15] flink git commit: [FLINK-2580] [runtime] Expose more methods form Hadoop output streams and exposes wrapped input and output streams.

Repository: flink
Updated Branches:
  refs/heads/master 18004343b -> 304139d48


[FLINK-2580] [runtime] Expose more methods form Hadoop output streams and exposes wrapped input and output streams.


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

Branch: refs/heads/master
Commit: 304139d48101d5c7eb9f0d5fa79cce37f235b0ab
Parents: b18e410
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 1 13:16:46 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200

----------------------------------------------------------------------
 .../flink/core/fs/FSDataOutputStream.java       |   7 +-
 .../core/fs/local/LocalDataOutputStream.java    |  11 ++
 .../runtime/fs/hdfs/HadoopDataInputStream.java  |  19 ++-
 .../runtime/fs/hdfs/HadoopDataOutputStream.java | 132 ++++++++++++++++++-
 .../flink/runtime/fs/hdfs/HadoopFileSystem.java |  22 +++-
 .../flink/runtime/fs/s3/S3DataOutputStream.java |  54 +++-----
 6 files changed, 191 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
index 831b29e..9974c9f 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FSDataOutputStream.java
@@ -16,14 +16,17 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.fs;
 
+import java.io.IOException;
 import java.io.OutputStream;
 
 /**
  * Interface for a data output stream to a file on a {@link FileSystem}.
- * 
  */
 public abstract class FSDataOutputStream extends OutputStream {
+
+	public abstract void flush() throws IOException;
+
+	public abstract void sync() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
index afef7c1..ea1ca34 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java
@@ -77,4 +77,15 @@ public class LocalDataOutputStream extends FSDataOutputStream {
 	public void close() throws IOException {
 		fos.close();
 	}
+
+
+	@Override
+	public void flush() throws IOException {
+		fos.flush();
+	}
+
+	@Override
+	public void sync() throws IOException {
+		fos.getFD().sync();
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
index 9a62885..6eeeb57 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataInputStream.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.fs.hdfs;
 
 import java.io.IOException;
@@ -26,11 +25,10 @@ import org.apache.flink.core.fs.FSDataInputStream;
 /**
  * Concrete implementation of the {@link FSDataInputStream} for the
  * Hadoop Distributed File System.
- * 
  */
 public final class HadoopDataInputStream extends FSDataInputStream {
 
-	private org.apache.hadoop.fs.FSDataInputStream fsDataInputStream = null;
+	private final org.apache.hadoop.fs.FSDataInputStream fsDataInputStream;
 
 	/**
 	 * Creates a new data input stream from the given HDFS input stream
@@ -39,13 +37,15 @@ public final class HadoopDataInputStream extends FSDataInputStream {
 	 *        the HDFS input stream
 	 */
 	public HadoopDataInputStream(org.apache.hadoop.fs.FSDataInputStream fsDataInputStream) {
+		if (fsDataInputStream == null) {
+			throw new NullPointerException();
+		}
 		this.fsDataInputStream = fsDataInputStream;
 	}
 
 
 	@Override
 	public synchronized void seek(long desired) throws IOException {
-
 		fsDataInputStream.seek(desired);
 	}
 
@@ -68,17 +68,22 @@ public final class HadoopDataInputStream extends FSDataInputStream {
 	public int read(byte[] buffer, int offset, int length) throws IOException {
 		return fsDataInputStream.read(buffer, offset, length);
 	}
-
-
+	
 	@Override
 	public int available() throws IOException {
 		return fsDataInputStream.available();
 	}
 
-
 	@Override
 	public long skip(long n) throws IOException {
 		return fsDataInputStream.skip(n);
 	}
 
+	/**
+	 * Gets the wrapped Hadoop input stream.
+	 * @return The wrapped Hadoop input stream.
+	 */
+	public org.apache.hadoop.fs.FSDataInputStream getHadoopInputStream() {
+		return fsDataInputStream;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
index 3f6cb1c..b0ff4b3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopDataOutputStream.java
@@ -16,24 +16,27 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.fs.hdfs;
 
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 
 import org.apache.flink.core.fs.FSDataOutputStream;
 
-public final class HadoopDataOutputStream extends FSDataOutputStream {
+public class HadoopDataOutputStream extends FSDataOutputStream {
 
-	private org.apache.hadoop.fs.FSDataOutputStream fdos;
+	private final org.apache.hadoop.fs.FSDataOutputStream fdos;
 
 	public HadoopDataOutputStream(org.apache.hadoop.fs.FSDataOutputStream fdos) {
+		if (fdos == null) {
+			throw new NullPointerException();
+		}
 		this.fdos = fdos;
 	}
 
 	@Override
 	public void write(int b) throws IOException {
-
 		fdos.write(b);
 	}
 
@@ -47,4 +50,125 @@ public final class HadoopDataOutputStream extends FSDataOutputStream {
 		fdos.close();
 	}
 
+	@Override
+	public void flush() throws IOException {
+		if (HFLUSH_METHOD != null) {
+			try {
+				HFLUSH_METHOD.invoke(fdos);
+			}
+			catch (InvocationTargetException e) {
+				Throwable cause = e.getTargetException();
+				if (cause instanceof IOException) {
+					throw (IOException) cause;
+				}
+				else if (cause instanceof RuntimeException) {
+					throw (RuntimeException) cause;
+				}
+				else if (cause instanceof Error) {
+					throw (Error) cause;
+				}
+				else {
+					throw new IOException("Exception while invoking hflush()", cause);
+				}
+			}
+			catch (IllegalAccessException e) {
+				throw new IOException("Cannot invoke hflush()", e);
+			}
+		}
+		else if (HFLUSH_ERROR != null) {
+			if (HFLUSH_ERROR instanceof NoSuchMethodException) {
+				throw new UnsupportedOperationException("hflush() method is not available in this version of Hadoop.");
+			}
+			else {
+				throw new IOException("Cannot access hflush() method", HFLUSH_ERROR);
+			}
+		}
+		else {
+			throw new UnsupportedOperationException("hflush() is not available in this version of Hadoop.");
+		}
+	}
+
+	@Override
+	public void sync() throws IOException {
+		if (HSYNC_METHOD != null) {
+			try {
+				HSYNC_METHOD.invoke(fdos);
+			}
+			catch (InvocationTargetException e) {
+				Throwable cause = e.getTargetException();
+				if (cause instanceof IOException) {
+					throw (IOException) cause;
+				}
+				else if (cause instanceof RuntimeException) {
+					throw (RuntimeException) cause;
+				}
+				else if (cause instanceof Error) {
+					throw (Error) cause;
+				}
+				else {
+					throw new IOException("Exception while invoking hsync()", cause);
+				}
+			}
+			catch (IllegalAccessException e) {
+				throw new IOException("Cannot invoke hsync()", e);
+			}
+		}
+		else if (HSYNC_ERROR != null) {
+			if (HSYNC_ERROR instanceof NoSuchMethodException) {
+				throw new UnsupportedOperationException("hsync() method is not available in this version of Hadoop.");
+			}
+			else {
+				throw new IOException("Cannot access hsync() method", HSYNC_ERROR);
+			}
+		}
+		else {
+			throw new UnsupportedOperationException("hsync() is not available in this version of Hadoop.");
+		}
+	}
+
+	/**
+	 * Gets the wrapped Hadoop output stream.
+	 * @return The wrapped Hadoop output stream.
+	 */
+	public org.apache.hadoop.fs.FSDataOutputStream getHadoopOutputStream() {
+		return fdos;
+	}
+	
+	// ------------------------------------------------------------------------
+	// utilities to bridge hsync and hflush to hadoop, even through it is not supported in Hadoop 1
+	// ------------------------------------------------------------------------
+	
+	private static final Method HFLUSH_METHOD;
+	private static final Method HSYNC_METHOD;
+	
+	private static final Throwable HFLUSH_ERROR;
+	private static final Throwable HSYNC_ERROR;
+	
+	static {
+		Method hflush = null;
+		Method hsync = null;
+
+		Throwable flushError = null;
+		Throwable syncError = null;
+		
+		try {
+			hflush = org.apache.hadoop.fs.FSDataOutputStream.class.getMethod("hflush");
+		}
+		catch (Throwable t) {
+			flushError = t;
+		}
+
+		try {
+			hsync = org.apache.hadoop.fs.FSDataOutputStream.class.getMethod("hsync");
+		}
+		catch (Throwable t) {
+			syncError = t;
+		}
+		
+		HFLUSH_METHOD = hflush;
+		HSYNC_METHOD = hsync;
+		
+		HFLUSH_ERROR = flushError;
+		HSYNC_ERROR = syncError;
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
index 152b2e8..4e05ebe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
@@ -23,18 +23,18 @@ import java.lang.reflect.Method;
 import java.net.URI;
 import java.net.UnknownHostException;
 
-import org.apache.flink.core.fs.HadoopFileSystemWrapper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
+
+import org.apache.flink.core.fs.HadoopFileSystemWrapper;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.GlobalConfiguration;
 import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.util.InstantiationUtil;
+
 import org.apache.hadoop.conf.Configuration;
 
 /**
@@ -264,6 +264,14 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
 		return fs.getUri();
 	}
 
+	/**
+	 * Gets the underlying Hadoop FileSystem.
+	 * @return The underlying Hadoop FileSystem.
+	 */
+	public org.apache.hadoop.fs.FileSystem getHadoopFileSystem() {
+		return this.fs;
+	}
+	
 	@Override
 	public void initialize(URI path) throws IOException {
 		
@@ -367,21 +375,21 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
 	}
 
 	@Override
-	public FSDataInputStream open(final Path f, final int bufferSize) throws IOException {
+	public HadoopDataInputStream open(final Path f, final int bufferSize) throws IOException {
 		final org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(f.toString());
 		final org.apache.hadoop.fs.FSDataInputStream fdis = this.fs.open(path, bufferSize);
 		return new HadoopDataInputStream(fdis);
 	}
 
 	@Override
-	public FSDataInputStream open(final Path f) throws IOException {
+	public HadoopDataInputStream open(final Path f) throws IOException {
 		final org.apache.hadoop.fs.Path path = new org.apache.hadoop.fs.Path(f.toString());
 		final org.apache.hadoop.fs.FSDataInputStream fdis = fs.open(path);
 		return new HadoopDataInputStream(fdis);
 	}
 
 	@Override
-	public FSDataOutputStream create(final Path f, final boolean overwrite, final int bufferSize,
+	public HadoopDataOutputStream create(final Path f, final boolean overwrite, final int bufferSize,
 			final short replication, final long blockSize)
 	throws IOException
 	{
@@ -392,7 +400,7 @@ public final class HadoopFileSystem extends FileSystem implements HadoopFileSyst
 
 
 	@Override
-	public FSDataOutputStream create(final Path f, final boolean overwrite) throws IOException {
+	public HadoopDataOutputStream create(final Path f, final boolean overwrite) throws IOException {
 		final org.apache.hadoop.fs.FSDataOutputStream fsDataOutputStream = this.fs
 			.create(new org.apache.hadoop.fs.Path(f.toString()), overwrite);
 		return new HadoopDataOutputStream(fsDataOutputStream);

http://git-wip-us.apache.org/repos/asf/flink/blob/304139d4/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
index 0501364..f830613 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/fs/s3/S3DataOutputStream.java
@@ -69,6 +69,7 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 	private int partNumber = 1; // First valid upload part number is 1.
 
 	private int bytesWritten = 0;
+	
 
 	private final class InternalUploadInputStream extends InputStream {
 
@@ -83,12 +84,9 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 			this.length = length;
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
+
 		@Override
 		public int read() throws IOException {
-
 			if (this.length - this.bytesRead == 0) {
 				return -1;
 			}
@@ -96,21 +94,14 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 			return (int) this.srcBuf[this.bytesRead++];
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
+
 		@Override
 		public int read(final byte[] buf) throws IOException {
-
 			return read(buf, 0, buf.length);
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
 		public int read(final byte[] buf, final int off, final int len) throws IOException {
-
 			if (this.length - this.bytesRead == 0) {
 				return -1;
 			}
@@ -122,18 +113,12 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 			return bytesToCopy;
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
 		public int available() throws IOException {
 
 			return (this.length - bytesRead);
 		}
 
-		/**
-		 * {@inheritDoc}
-		 */
 		@Override
 		public long skip(final long n) throws IOException {
 
@@ -159,7 +144,6 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 
 	@Override
 	public void write(final int b) throws IOException {
-
 		// Upload buffer to S3
 		if (this.bytesWritten == this.buf.length) {
 			uploadPartAndFlushBuffer();
@@ -171,11 +155,9 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 
 	@Override
 	public void write(final byte[] b, final int off, final int len) throws IOException {
-
 		int nextPos = off;
 
 		while (nextPos < len) {
-
 			// Upload buffer to S3
 			if (this.bytesWritten == this.buf.length) {
 				uploadPartAndFlushBuffer();
@@ -191,14 +173,12 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 
 	@Override
 	public void write(final byte[] b) throws IOException {
-
 		write(b, 0, b.length);
 	}
 
 
 	@Override
 	public void close() throws IOException {
-
 		if (this.uploadId == null) {
 			// This is not a multipart upload
 
@@ -251,10 +231,13 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 		}
 	}
 
-
+	@Override
+	public void sync() throws IOException {
+		// can do nothing here
+	}
+	
 	@Override
 	public void flush() throws IOException {
-
 		// Flush does nothing in this implementation since we ways have to transfer at least 5 MB in a multipart upload
 	}
 
@@ -267,7 +250,6 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 		}
 
 		try {
-
 			if (this.partNumber >= MAX_PART_NUMBER) {
 				throw new IOException("Cannot upload any more data: maximum part number reached");
 			}
@@ -287,9 +269,11 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 			this.bytesWritten = 0;
 			operationSuccessful = true;
 
-		} catch (AmazonServiceException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		} finally {
+		}
+		catch (AmazonServiceException e) {
+			throw new IOException(e.getMessage(), e);
+		}
+		finally {
 			if (!operationSuccessful) {
 				abortUpload();
 			}
@@ -312,9 +296,11 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 			operationSuccessful = true;
 			return result.getUploadId();
 
-		} catch (AmazonServiceException e) {
-			throw new IOException(StringUtils.stringifyException(e));
-		} finally {
+		}
+		catch (AmazonServiceException e) {
+			throw new IOException(e.getMessage(), e);
+		}
+		finally {
 			if (!operationSuccessful) {
 				abortUpload();
 			}
@@ -322,7 +308,6 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 	}
 
 	private void abortUpload() {
-
 		if (this.uploadId == null) {
 			// This is not a multipart upload, nothing to do here
 			return;
@@ -332,7 +317,8 @@ public final class S3DataOutputStream extends FSDataOutputStream {
 			final AbortMultipartUploadRequest request = new AbortMultipartUploadRequest(this.bucket, this.object,
 				this.uploadId);
 			this.s3Client.abortMultipartUpload(request);
-		} catch (AmazonServiceException e) {
+		}
+		catch (AmazonServiceException e) {
 			// Ignore exception
 		}
 	}


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
index 62c2013..f40171a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorterTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.sort;
 
 import java.util.List;
@@ -25,9 +24,8 @@ import java.util.Random;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
-import org.apache.flink.runtime.operators.sort.QuickSort;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.RandomIntPairGenerator;
 import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
@@ -43,15 +41,15 @@ import org.junit.Test;
 /**
  * 
  */
-public class FixedLengthRecordSorterTest
-{
+public class FixedLengthRecordSorterTest {
+	
 	private static final long SEED = 649180756312423613L;
 
 	private static final int MEMORY_SIZE = 1024 * 1024 * 64;
 	
 	private static final int MEMORY_PAGE_SIZE = 32 * 1024; 
 
-	private DefaultMemoryManager memoryManager;
+	private MemoryManager memoryManager;
 	
 	private TypeSerializer<IntPair> serializer;
 	
@@ -60,7 +58,7 @@ public class FixedLengthRecordSorterTest
 
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, MEMORY_PAGE_SIZE);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
 		this.serializer = new IntPairSerializer();
 		this.comparator = new IntPairComparator();
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
index 429062b..e4dbfd3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerITCase.java
@@ -39,16 +39,18 @@ import org.apache.flink.api.java.typeutils.ValueTypeInfo;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.FileChannelOutputView;
 import org.apache.flink.runtime.io.disk.SeekableFileChannelInputView;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.types.Value;
 import org.apache.flink.util.MutableObjectIterator;
+
 import org.junit.Test;
 
 public class LargeRecordHandlerITCase {
@@ -62,7 +64,7 @@ public class LargeRecordHandlerITCase {
 		final int NUM_RECORDS = 10;
 		
 		try {
-			final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+			final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 			final AbstractInvokable owner = new DummyInvokable();
 			
 			final List<MemorySegment> initialMemory = memMan.allocatePages(owner, 6);
@@ -143,9 +145,7 @@ public class LargeRecordHandlerITCase {
 			fail(e.getMessage());
 		}
 		finally {
-			if (ioMan != null) {
-				ioMan.shutdown();
-			}
+			ioMan.shutdown();
 		}
 	}
 	
@@ -155,8 +155,6 @@ public class LargeRecordHandlerITCase {
 
 		private static final byte[] BUFFER = new byte[50000000];
 		
-//		private static final byte[] BUFFER = new byte[500000000];
-		
 		static {
 			for (int i = 0; i < BUFFER.length; i++) {
 				BUFFER[i] = (byte) i;
@@ -178,9 +176,9 @@ public class LargeRecordHandlerITCase {
 		@Override
 		public void read(DataInputView in) throws IOException {
 			val = in.readInt();
-			for (int i = 0; i < BUFFER.length; i++) {
+			for (byte bufferByte : BUFFER) {
 				byte b = in.readByte();
-				assertEquals(BUFFER[i], b);
+				assertEquals(bufferByte, b);
 			}
 		}
 		
@@ -192,7 +190,7 @@ public class LargeRecordHandlerITCase {
 	}
 	
 	
-//	@Test
+	@Test
 	public void fileTest() {
 		
 		final IOManager ioMan = new IOManagerAsync();
@@ -203,7 +201,7 @@ public class LargeRecordHandlerITCase {
 		FileIOChannel.ID channel = null;
 		
 		try {
-			final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+			final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 			final AbstractInvokable owner = new DummyInvokable();
 			
 			final List<MemorySegment> memory = memMan.allocatePages(owner, NUM_PAGES);
@@ -266,12 +264,10 @@ public class LargeRecordHandlerITCase {
 			if (channel != null) {
 				try {
 					ioMan.deleteChannel(channel);
-				} catch (IOException e) {}
-			}
-			
-			if (ioMan != null) {
-				ioMan.shutdown();
+				} catch (IOException ignored) {}
 			}
+
+			ioMan.shutdown();
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
index 423ff9a..49e1e76 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/LargeRecordHandlerTest.java
@@ -31,10 +31,11 @@ import org.apache.flink.api.java.tuple.Tuple3;
 import org.apache.flink.api.java.typeutils.TupleTypeInfo;
 import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.util.MutableObjectIterator;
 import org.junit.Test;
@@ -49,7 +50,7 @@ public class LargeRecordHandlerTest {
 		final int NUM_PAGES = 50;
 		
 		try {
-			final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+			final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 			final AbstractInvokable owner = new DummyInvokable();
 			final List<MemorySegment> memory = memMan.allocatePages(owner, NUM_PAGES);
 			
@@ -86,9 +87,7 @@ public class LargeRecordHandlerTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (ioMan != null) {
-				ioMan.shutdown();
-			}
+			ioMan.shutdown();
 		}
 	}
 	
@@ -101,7 +100,7 @@ public class LargeRecordHandlerTest {
 		final int NUM_RECORDS = 25000;
 		
 		try {
-			final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+			final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 			final AbstractInvokable owner = new DummyInvokable();
 			
 			final List<MemorySegment> initialMemory = memMan.allocatePages(owner, 6);
@@ -174,9 +173,7 @@ public class LargeRecordHandlerTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (ioMan != null) {
-				ioMan.shutdown();
-			}
+			ioMan.shutdown();
 		}
 	}
 	
@@ -189,7 +186,7 @@ public class LargeRecordHandlerTest {
 		final int NUM_RECORDS = 25000;
 		
 		try {
-			final DefaultMemoryManager memMan = new DefaultMemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, true);
+			final MemoryManager memMan = new MemoryManager(NUM_PAGES * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 			final AbstractInvokable owner = new DummyInvokable();
 			
 			final List<MemorySegment> initialMemory = memMan.allocatePages(owner, 6);
@@ -264,9 +261,7 @@ public class LargeRecordHandlerTest {
 			fail(e.getMessage());
 		}
 		finally {
-			if (ioMan != null) {
-				ioMan.shutdown();
-			}
+			ioMan.shutdown();
 		}
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
index 6548052..7f6db00 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeInnerJoinIteratorITCase.java
@@ -32,9 +32,12 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.testutils.*;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.runtime.operators.testutils.Match;
+import org.apache.flink.runtime.operators.testutils.MatchRemovingJoiner;
+import org.apache.flink.runtime.operators.testutils.TestData;
 import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator;
 import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
 import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
@@ -45,7 +48,11 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 
 @SuppressWarnings("deprecation")
@@ -97,7 +104,7 @@ public class NonReusingSortMergeInnerJoinIteratorITCase {
 				new TypeSerializer<?>[] { IntSerializer.INSTANCE });
 		pairComparator = new GenericPairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>>(comparator1, comparator2);
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
index 91609bb..7272595 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NonReusingSortMergeOuterJoinIteratorITCase.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.util.MutableObjectIterator;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
index abead64..3e941dd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorterTest.java
@@ -25,9 +25,8 @@ import java.util.Random;
 import org.apache.flink.api.common.typeutils.record.RecordComparator;
 import org.apache.flink.api.common.typeutils.record.RecordSerializer;
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;
-import org.apache.flink.runtime.operators.sort.QuickSort;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.TestData;
 import org.apache.flink.runtime.operators.testutils.TestData.Key;
@@ -57,12 +56,12 @@ public class NormalizedKeySorterTest
 	
 	private static final int MEMORY_PAGE_SIZE = 32 * 1024; 
 
-	private DefaultMemoryManager memoryManager;
+	private MemoryManager memoryManager;
 
 
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, MEMORY_PAGE_SIZE);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
 	}
 
 	@After
@@ -369,8 +368,7 @@ public class NormalizedKeySorterTest
 	}
 	
 	@Test
-	public void testSortShortStringKeys() throws Exception
-	{
+	public void testSortShortStringKeys() throws Exception {
 		final int numSegments = MEMORY_SIZE / MEMORY_PAGE_SIZE;
 		final List<MemorySegment> memory = this.memoryManager.allocatePages(new DummyInvokable(), numSegments);
 		
@@ -418,8 +416,7 @@ public class NormalizedKeySorterTest
 	}
 	
 	@Test
-	public void testSortLongStringKeys() throws Exception
-	{
+	public void testSortLongStringKeys() throws Exception {
 		final int numSegments = MEMORY_SIZE / MEMORY_PAGE_SIZE;
 		final List<MemorySegment> memory = this.memoryManager.allocatePages(new DummyInvokable(), numSegments);
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
index 39316e3..d92be18 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeInnerJoinIteratorITCase.java
@@ -32,9 +32,12 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.testutils.*;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.runtime.operators.testutils.Match;
+import org.apache.flink.runtime.operators.testutils.MatchRemovingJoiner;
+import org.apache.flink.runtime.operators.testutils.TestData;
 import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator;
 import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.KeyMode;
 import org.apache.flink.runtime.operators.testutils.TestData.TupleGenerator.ValueMode;
@@ -45,7 +48,11 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
 
 @SuppressWarnings("deprecation")
@@ -97,7 +104,7 @@ public class ReusingSortMergeInnerJoinIteratorITCase {
 				new TypeSerializer<?>[] { IntSerializer.INSTANCE });
 		pairComparator = new GenericPairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>>(comparator1, comparator2);
 
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
index 779cf37..2cec393 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ReusingSortMergeOuterJoinIteratorITCase.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.util.MutableObjectIterator;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
index a085eeb..ece20ff 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java
@@ -29,8 +29,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.PactDriver;
 import org.apache.flink.runtime.operators.PactTaskContext;
 import org.apache.flink.runtime.operators.ResettablePactDriver;
@@ -100,7 +99,7 @@ public class BinaryOperatorTestBase<S extends Function, IN, OUT> extends TestLog
 		this.perSortMem = perSortMemory;
 		this.perSortFractionMem = (double) perSortMemory / totalMem;
 		this.ioManager = new IOManagerAsync();
-		this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem, 1) : null;
+		this.memManager = totalMem > 0 ? new MemoryManager(totalMem, 1) : null;
 		
 		this.inputs = new ArrayList<>();
 		this.comparators = new ArrayList<>();

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
index 12ca909..1737349 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java
@@ -37,8 +37,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.PactDriver;
 import org.apache.flink.runtime.operators.PactTaskContext;
 import org.apache.flink.runtime.operators.ResettablePactDriver;
@@ -104,7 +103,7 @@ public class DriverTestBase<S extends Function> extends TestLogger implements Pa
 		this.perSortMem = perSortMemory;
 		this.perSortFractionMem = (double)perSortMemory/totalMem;
 		this.ioManager = new IOManagerAsync();
-		this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem,1) : null;
+		this.memManager = totalMem > 0 ? new MemoryManager(totalMem,1) : null;
 
 		this.inputs = new ArrayList<MutableObjectIterator<Record>>();
 		this.comparators = new ArrayList<TypeComparator<Record>>();

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
index 5fc9bb9..972aeda 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/MockEnvironment.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.operators.testutils;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.execution.Environment;
@@ -39,8 +39,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.types.Record;
@@ -94,7 +93,7 @@ public class MockEnvironment implements Environment {
 		this.inputs = new LinkedList<InputGate>();
 		this.outputs = new LinkedList<ResultPartitionWriter>();
 
-		this.memManager = new DefaultMemoryManager(memorySize, 1);
+		this.memManager = new MemoryManager(memorySize, 1);
 		this.ioManager = new IOManagerAsync();
 		this.inputSplitProvider = inputSplitProvider;
 		this.bufferSize = bufferSize;
@@ -125,7 +124,7 @@ public class MockEnvironment implements Environment {
 
 				@Override
 				public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
-					return new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
+					return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), mock(BufferRecycler.class));
 				}
 			});
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
index c04d051..4662762 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/TaskTestBase.java
@@ -29,7 +29,7 @@ import org.apache.flink.core.fs.FileSystem.WriteMode;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.io.network.partition.consumer.IteratorWrappingTestSingleInputGate;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.PactDriver;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;
 import org.apache.flink.runtime.operators.util.TaskConfig;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
index 9600e5a..924a16b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java
@@ -29,8 +29,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.PactDriver;
 import org.apache.flink.runtime.operators.PactTaskContext;
 import org.apache.flink.runtime.operators.ResettablePactDriver;
@@ -106,7 +105,7 @@ public class UnaryOperatorTestBase<S extends Function, IN, OUT> extends TestLogg
 		this.perSortMem = perSortMemory;
 		this.perSortFractionMem = (double)perSortMemory/totalMem;
 		this.ioManager = new IOManagerAsync();
-		this.memManager = totalMem > 0 ? new DefaultMemoryManager(totalMem,1) : null;
+		this.memManager = totalMem > 0 ? new MemoryManager(totalMem, 1) : null;
 		this.owner = new DummyInvokable();
 
 		Configuration config = new Configuration();

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
index cbbeca0..8fbd4f7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/BloomFilterTest.java
@@ -18,7 +18,10 @@
 
 package org.apache.flink.runtime.operators.util;
 
+import org.apache.flink.core.memory.HeapMemorySegment;
 import org.apache.flink.core.memory.MemorySegment;
+
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -37,7 +40,7 @@ public class BloomFilterTest {
 		int bitsSize = BloomFilter.optimalNumOfBits(INPUT_SIZE, FALSE_POSITIVE_PROBABILITY);
 		bitsSize = bitsSize + (Long.SIZE - (bitsSize % Long.SIZE));
 		int byteSize = bitsSize >>> 3;
-		MemorySegment memorySegment = new MemorySegment(new byte[byteSize]);
+		MemorySegment memorySegment = MemorySegmentFactory.allocateUnpooledSegment(byteSize);
 		bloomFilter = new BloomFilter(INPUT_SIZE, byteSize);
 		bloomFilter.setBitsLocation(memorySegment, 0);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
index 7debb08..1060e55 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/util/HashVsSortMiniBenchmark.java
@@ -25,11 +25,11 @@ import org.apache.flink.api.common.typeutils.record.RecordComparator;
 import org.apache.flink.api.common.typeutils.record.RecordPairComparator;
 import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
 import org.apache.flink.api.java.record.functions.JoinFunction;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator;
 import org.apache.flink.runtime.operators.hash.ReusingBuildSecondHashMatchIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeInnerJoinIterator;
@@ -94,7 +94,7 @@ public class HashVsSortMiniBenchmark {
 		this.comparator2 = new RecordComparator(new int[] {0}, new Class[] {TestData.Key.class});
 		this.pairComparator11 = new RecordPairComparator(new int[] {0}, new int[] {0}, new Class[] {TestData.Key.class});
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, PAGE_SIZE);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
index 6d743d5..7857123 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java
@@ -39,7 +39,7 @@ import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointNotificationOperator;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointedOperator;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 
 import org.junit.Before;
 import org.junit.Test;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
index 0bb4063..0a7f08d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskManagerComponentsStartupShutdownTest.java
@@ -27,6 +27,7 @@ import akka.actor.Props;
 import akka.testkit.JavaTestKit;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.StreamingMode;
 import org.apache.flink.runtime.akka.AkkaUtils;
 import org.apache.flink.runtime.instance.InstanceConnectionInfo;
@@ -37,8 +38,7 @@ import org.apache.flink.runtime.io.network.netty.NettyConfig;
 import org.apache.flink.runtime.jobmanager.JobManager;
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService;
 import org.apache.flink.runtime.leaderretrieval.StandaloneLeaderRetrievalService;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 
 import org.apache.flink.runtime.messages.TaskManagerMessages;
 import org.apache.flink.runtime.testingUtils.TestingUtils;
@@ -87,12 +87,12 @@ public class TaskManagerComponentsStartupShutdownTest {
 					config);
 
 			final NetworkEnvironmentConfiguration netConf = new NetworkEnvironmentConfiguration(
-					32, BUFFER_SIZE, IOManager.IOMode.SYNC, Option.<NettyConfig>empty(),
+					32, BUFFER_SIZE, MemoryType.HEAP, IOManager.IOMode.SYNC, Option.<NettyConfig>empty(),
 					new Tuple2<Integer, Integer>(0, 0));
 
 			final InstanceConnectionInfo connectionInfo = new InstanceConnectionInfo(InetAddress.getLocalHost(), 10000);
 
-			final MemoryManager memManager = new DefaultMemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, false);
+			final MemoryManager memManager = new MemoryManager(32 * BUFFER_SIZE, 1, BUFFER_SIZE, MemoryType.HEAP, false);
 			final IOManager ioManager = new IOManagerAsync(TMP_DIR);
 			final NetworkEnvironment network = new NetworkEnvironment(
 				TestingUtils.defaultExecutionContext(),

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
index 5cfc2a2..cf1bfc5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java
@@ -42,7 +42,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.TaskMessages;
 
 import org.junit.After;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
index ac6744e..3631ba1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/DataInputOutputSerializerTest.java
@@ -16,9 +16,9 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.util;
 
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.junit.Assert;
 
 import org.apache.flink.core.memory.MemorySegment;
@@ -38,7 +38,7 @@ public class DataInputOutputSerializerTest {
 		SerializationTestType randomInt = Util.randomRecord(SerializationTestTypeFactory.INT);
 
 		DataOutputSerializer serializer = new DataOutputSerializer(randomInt.length());
-		MemorySegment segment = new MemorySegment(new byte[randomInt.length()]);
+		MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(randomInt.length());
 
 		try {
 			// empty buffer, read buffer should be empty

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
index 475115e..8746fbb 100644
--- a/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
+++ b/flink-runtime/src/test/scala/org/apache/flink/runtime/testingUtils/TestingTaskManager.scala
@@ -25,7 +25,7 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
+import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.messages.JobManagerMessages.{ResponseLeaderSessionID,
 RequestLeaderSessionID}
 import org.apache.flink.runtime.messages.Messages.{Acknowledge, Disconnect}
@@ -53,7 +53,7 @@ import scala.language.postfixOps
 class TestingTaskManager(
     config: TaskManagerConfiguration,
     connectionInfo: InstanceConnectionInfo,
-    memoryManager: DefaultMemoryManager,
+    memoryManager: MemoryManager,
     ioManager: IOManager,
     network: NetworkEnvironment,
     numberOfSlots: Int,

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
index 2bad197..cabed14 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/BufferSpiller.java
@@ -27,6 +27,7 @@ import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
@@ -336,7 +337,7 @@ public class BufferSpiller {
 							"Spilled buffer (%d bytes) is larger than page size of (%d bytes)", length, pageSize));
 				}
 
-				MemorySegment seg = new MemorySegment(new byte[pageSize]);
+				MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(pageSize);
 				
 				int segPos = 0;
 				int bytesRemaining = length;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
index 173e894..b78ec44 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
@@ -20,7 +20,8 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
@@ -84,7 +85,7 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 		for (int i = 0; i < numInputChannels; i++) {
 			final int channelIndex = i;
 			final RecordSerializer<SerializationDelegate<Object>> recordSerializer = new SpanningRecordSerializer<SerializationDelegate<Object>>();
-			final SerializationDelegate<Object> delegate = new SerializationDelegate(new MultiplexingStreamRecordSerializer<T>(serializer));
+			final SerializationDelegate<Object> delegate = new SerializationDelegate<Object>(new MultiplexingStreamRecordSerializer<T>(serializer));
 
 			inputQueues[channelIndex] = new ConcurrentLinkedQueue<InputValue<Object>>();
 			inputChannels[channelIndex] = new TestInputChannel(inputGate, i);
@@ -98,20 +99,25 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 						when(inputChannels[channelIndex].getInputChannel().isReleased()).thenReturn(
 								true);
 						return EventSerializer.toBuffer(EndOfPartitionEvent.INSTANCE);
-					} else if (input != null && input.isStreamRecord()) {
+					}
+					else if (input != null && input.isStreamRecord()) {
 						Object inputElement = input.getStreamRecord();
-						final Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]),
+						final Buffer buffer = new Buffer(
+								MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
 								mock(BufferRecycler.class));
+						
 						recordSerializer.setNextBuffer(buffer);
 						delegate.setInstance(inputElement);
 						recordSerializer.addRecord(delegate);
 
 						// Call getCurrentBuffer to ensure size is set
 						return recordSerializer.getCurrentBuffer();
-					} else if (input != null && input.isEvent()) {
+					}
+					else if (input != null && input.isEvent()) {
 						AbstractEvent event = input.getEvent();
 						return EventSerializer.toBuffer(event);
-					} else {
+					}
+					else {
 						synchronized (inputQueues[channelIndex]) {
 							inputQueues[channelIndex].wait();
 							return answer(invocationOnMock);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
index 678b145..a8a989b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferMassiveRandomTest.java
@@ -22,6 +22,7 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.util.Random;
 
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.event.TaskEvent;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
@@ -50,8 +51,8 @@ public class BarrierBufferMassiveRandomTest {
 		try {
 			ioMan = new IOManagerAsync();
 			
-			BufferPool pool1 = new NetworkBufferPool(100, PAGE_SIZE).createBufferPool(100, true);
-			BufferPool pool2 = new NetworkBufferPool(100, PAGE_SIZE).createBufferPool(100, true);
+			BufferPool pool1 = new NetworkBufferPool(100, PAGE_SIZE, MemoryType.HEAP).createBufferPool(100, true);
+			BufferPool pool2 = new NetworkBufferPool(100, PAGE_SIZE, MemoryType.HEAP).createBufferPool(100, true);
 
 			RandomGeneratingInputGate myIG = new RandomGeneratingInputGate(
 					new BufferPool[] { pool1, pool2 },

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
index a95839a..d4fdc59 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierBufferTest.java
@@ -18,7 +18,8 @@
 
 package org.apache.flink.streaming.runtime.io;
 
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
@@ -888,7 +889,10 @@ public class BarrierBufferTest {
 	private static BufferOrEvent createBuffer(int channel) {
 		// since we have no access to the contents, we need to use the size as an
 		// identifier to validate correctness here
-		Buffer buf = new Buffer(new MemorySegment(new byte[PAGE_SIZE]), FreeingBufferRecycler.INSTANCE);
+		Buffer buf = new Buffer(
+				MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE),
+				FreeingBufferRecycler.INSTANCE);
+		
 		buf.setSize(SIZE_COUNTER++);
 		return new BufferOrEvent(buf, channel);
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
index c6010d6..b9b6e5f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BarrierTrackerTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.streaming.runtime.io;
 
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
 import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
@@ -339,7 +339,7 @@ public class BarrierTrackerTest {
 
 	private static BufferOrEvent createBuffer(int channel) {
 		return new BufferOrEvent(
-				new Buffer(new MemorySegment(new byte[] { 1, 2 }), FreeingBufferRecycler.INSTANCE), channel);
+				new Buffer(MemorySegmentFactory.wrap(new byte[]{1, 2}), FreeingBufferRecycler.INSTANCE), channel);
 	}
 	
 	// ------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
index 355b7c8..e85eddb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/BufferSpillerTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -347,7 +348,7 @@ public class BufferSpillerTest {
 	}
 
 	private static BufferOrEvent generateRandomBuffer(int size, int channelIndex) {
-		MemorySegment seg = new MemorySegment(new byte[PAGE_SIZE]);
+		MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(PAGE_SIZE);
 		for (int i = 0; i < size; i++) {
 			seg.put(i, (byte) i);
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
index 7cc1958..45bbbda 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/io/StreamRecordWriterTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.streaming.runtime.io;
 
 import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.api.writer.ChannelSelector;
 import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.api.writer.RoundRobinChannelSelector;
@@ -33,7 +33,6 @@ import org.junit.runner.RunWith;
 
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
-
 import org.powermock.core.classloader.annotations.PrepareForTest;
 import org.powermock.modules.junit4.PowerMockRunner;
 
@@ -95,7 +94,9 @@ public class StreamRecordWriterTest {
 		when(mockProvider.requestBufferBlocking()).thenAnswer(new Answer<Buffer>() {
 			@Override
 			public Buffer answer(InvocationOnMock invocation) {
-				return new Buffer(new MemorySegment(new byte[4096]), FreeingBufferRecycler.INSTANCE);
+				return new Buffer(
+						MemorySegmentFactory.allocateUnpooledSegment(4096),
+						FreeingBufferRecycler.INSTANCE);
 			}
 		});
 		

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
index 9091fa7..4fec118 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamMockEnvironment.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.UnmodifiableConfiguration;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.broadcast.BroadcastVariableManager;
 import org.apache.flink.runtime.event.AbstractEvent;
@@ -40,8 +40,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.runtime.plugable.DeserializationDelegate;
 import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
@@ -95,7 +94,7 @@ public class StreamMockEnvironment implements Environment {
 		this.inputs = new LinkedList<InputGate>();
 		this.outputs = new LinkedList<ResultPartitionWriter>();
 
-		this.memManager = new DefaultMemoryManager(memorySize, 1);
+		this.memManager = new MemoryManager(memorySize, 1);
 		this.ioManager = new IOManagerAsync();
 		this.inputSplitProvider = inputSplitProvider;
 		this.bufferSize = bufferSize;
@@ -117,7 +116,9 @@ public class StreamMockEnvironment implements Environment {
 
 				@Override
 				public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
-					return new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
+					return new Buffer(
+							MemorySegmentFactory.allocateUnpooledSegment(bufferSize),
+							mock(BufferRecycler.class));
 				}
 			});
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index 987d0bc..96dbeab 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -24,7 +24,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.network.partition.consumer.StreamTestSingleInputGate;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
 import org.apache.flink.streaming.api.collector.selector.BroadcastOutputSelectorWrapper;
 import org.apache.flink.streaming.api.collector.selector.OutputSelector;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
index 889255c..b61a9b6 100644
--- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
+++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezRuntimeEnvironment.java
@@ -18,22 +18,19 @@
 
 package org.apache.flink.tez.runtime;
 
-
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 
 public class TezRuntimeEnvironment {
-
-	private static int DEFAULT_PAGE_SIZE = 32768;
-
+	
 	private final IOManager ioManager;
 
 	private final MemoryManager memoryManager;
 
 	public TezRuntimeEnvironment(long totalMemory) {
-		this.memoryManager = new DefaultMemoryManager(totalMemory, 1, DefaultMemoryManager.DEFAULT_PAGE_SIZE, true);
+		this.memoryManager = new MemoryManager(totalMemory, 1, MemoryManager.DEFAULT_PAGE_SIZE, MemoryType.HEAP, true);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
index 47fbad7..a745177 100644
--- a/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
+++ b/flink-staging/flink-tez/src/main/java/org/apache/flink/tez/runtime/TezTask.java
@@ -35,7 +35,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.PactDriver;
 import org.apache.flink.runtime.operators.PactTaskContext;
 import org.apache.flink.runtime.operators.shipping.ShipStrategyType;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java b/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
index 493de07..cb20e72 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/HashTableRecordWidthCombinations.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.operators.hash.MutableHashTable;
@@ -183,7 +184,7 @@ public class HashTableRecordWidthCombinations {
 	private static List<MemorySegment> getMemory(int numSegments, int segmentSize) {
 		ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(numSegments);
 		for (int i = 0; i < numSegments; i++) {
-			list.add(new MemorySegment(new byte[segmentSize]));
+			list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize));
 		}
 		return list;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
index 2da29e8..c11b93c 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringSorting.java
@@ -27,7 +27,6 @@ import java.io.IOException;
 import java.util.Random;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.StringComparator;
@@ -38,8 +37,7 @@ import org.apache.flink.api.java.typeutils.TypeInfoParser;
 import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.util.MutableObjectIterator;
@@ -83,7 +81,7 @@ public class MassiveStringSorting {
 			BufferedReader verifyReader = null;
 			
 			try {
-				MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
 					
 				TypeSerializer<String> serializer = StringSerializer.INSTANCE;
@@ -173,7 +171,7 @@ public class MassiveStringSorting {
 			BufferedReader verifyReader = null;
 			
 			try {
-				MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
 					
 				TupleTypeInfo<Tuple2<String, String[]>> typeInfo = (TupleTypeInfo<Tuple2<String, String[]>>) 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
index bc65ab9..7a484e7 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/manual/MassiveStringValueSorting.java
@@ -37,8 +37,7 @@ import org.apache.flink.api.java.typeutils.runtime.CopyableValueSerializer;
 import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.types.StringValue;
@@ -82,7 +81,7 @@ public class MassiveStringValueSorting {
 			BufferedReader verifyReader = null;
 			
 			try {
-				MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
 					
 				TypeSerializer<StringValue> serializer = new CopyableValueSerializer<StringValue>(StringValue.class);
@@ -129,9 +128,11 @@ public class MassiveStringValueSorting {
 		}
 		finally {
 			if (input != null) {
+				//noinspection ResultOfMethodCallIgnored
 				input.delete();
 			}
 			if (sorted != null) {
+				//noinspection ResultOfMethodCallIgnored
 				sorted.delete();
 			}
 		}
@@ -173,7 +174,7 @@ public class MassiveStringValueSorting {
 			BufferedReader verifyReader = null;
 			
 			try {
-				MemoryManager mm = new DefaultMemoryManager(1024 * 1024, 1);
+				MemoryManager mm = new MemoryManager(1024 * 1024, 1);
 				IOManager ioMan = new IOManagerAsync();
 					
 				TupleTypeInfo<Tuple2<StringValue, StringValue[]>> typeInfo = (TupleTypeInfo<Tuple2<StringValue, StringValue[]>>)
@@ -251,9 +252,11 @@ public class MassiveStringValueSorting {
 		}
 		finally {
 			if (input != null) {
+				//noinspection ResultOfMethodCallIgnored
 				input.delete();
 			}
 			if (sorted != null) {
+				//noinspection ResultOfMethodCallIgnored
 				sorted.delete();
 			}
 		}
@@ -348,7 +351,9 @@ public class MassiveStringValueSorting {
 				wrt.newLine();
 			}
 		} finally {
-			wrt.close();
+			if (wrt != null) {
+				wrt.close();
+			}
 		}
 		
 		return f;
@@ -388,7 +393,9 @@ public class MassiveStringValueSorting {
 				wrt.newLine();
 			}
 		} finally {
-			wrt.close();
+			if (wrt != null) {
+				wrt.close();
+			}
 		}
 		
 		return f;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
index 157aa0d..dd27eb5 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/misc/MassiveCaseClassSortingITCase.scala
@@ -28,7 +28,7 @@ import java.io.BufferedReader
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync
 import java.io.FileReader
 import org.apache.flink.util.MutableObjectIterator
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
+import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.common.typeutils.CompositeType
 import org.apache.flink.runtime.operators.sort.UnilateralSortMerger
@@ -89,7 +89,7 @@ class MassiveCaseClassSortingITCase {
           0,
           new ExecutionConfig)
         
-        val mm = new DefaultMemoryManager(1024 * 1024, 1)
+        val mm = new MemoryManager(1024 * 1024, 1)
         val ioMan = new IOManagerAsync()
         
         sorter = new UnilateralSortMerger[StringTuple](mm, ioMan, inputIterator,

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
index 21c6581..1b48612 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/runtime/CaseClassComparatorTest.scala
@@ -18,25 +18,19 @@
 
 package org.apache.flink.api.scala.runtime
 
+import java.util
+import java.util.{ArrayList, List, Random}
+
 import org.apache.flink.api.common.ExecutionConfig
-import org.junit.Test
-import org.junit.Assert._
-import org.apache.flink.api.scala._
-import org.apache.flink.api.common.typeutils.TypeSerializer
 import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.common.typeutils.CompositeType
-import org.apache.flink.core.memory.DataInputView
-import java.io.IOException
-import org.apache.flink.api.common.typeutils.TypeComparator
-import com.amazonaws.services.sqs.model.UnsupportedOperationException
-import org.apache.flink.core.memory.MemorySegment
-import org.apache.flink.core.memory.DataOutputView
+import org.apache.flink.api.common.typeutils.{CompositeType, TypeComparator}
+import org.apache.flink.api.scala._
+import org.apache.flink.core.memory._
+import org.apache.flink.runtime.operators.sort.{NormalizedKeySorter, QuickSort}
+
+import org.junit.Assert._
+import org.junit.Test
 import org.mockito.Mockito
-import org.apache.flink.runtime.operators.sort.NormalizedKeySorter
-import java.util.List
-import java.util.ArrayList
-import org.apache.flink.runtime.operators.sort.QuickSort
-import java.util.Random
 
 class CaseClassComparatorTest {
 
@@ -78,9 +72,9 @@ class CaseClassComparatorTest {
       
       
       val numMemSegs = 20
-      val memory : List[MemorySegment] = new ArrayList[MemorySegment](numMemSegs)
+      val memory : util.List[MemorySegment] = new util.ArrayList[MemorySegment](numMemSegs)
       for (i <- 1 to numMemSegs) {
-        memory.add(new MemorySegment(new Array[Byte](32*1024)))
+        memory.add(MemorySegmentFactory.allocateUnpooledSegment(32*1024))
       }
       
       val sorter : NormalizedKeySorter[CaseTestClass] = new NormalizedKeySorter[CaseTestClass](

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
index b95eb86..c9e193f 100644
--- a/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
+++ b/flink-yarn/src/main/scala/org/apache/flink/yarn/YarnTaskManager.scala
@@ -22,7 +22,7 @@ import org.apache.flink.runtime.instance.InstanceConnectionInfo
 import org.apache.flink.runtime.io.disk.iomanager.IOManager
 import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalService
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager
+import org.apache.flink.runtime.memory.MemoryManager
 import org.apache.flink.runtime.taskmanager.{NetworkEnvironmentConfiguration, TaskManagerConfiguration, TaskManager}
 import org.apache.flink.yarn.Messages.StopYarnSession
 
@@ -32,7 +32,7 @@ import org.apache.flink.yarn.Messages.StopYarnSession
 class YarnTaskManager(
     config: TaskManagerConfiguration,
     connectionInfo: InstanceConnectionInfo,
-    memoryManager: DefaultMemoryManager,
+    memoryManager: MemoryManager,
     ioManager: IOManager,
     network: NetworkEnvironment,
     numberOfSlots: Int,

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cdee6f6..7f17396 100644
--- a/pom.xml
+++ b/pom.xml
@@ -803,7 +803,7 @@ under the License.
 			<plugin>
 				<groupId>org.apache.maven.plugins</groupId>
 				<artifactId>maven-checkstyle-plugin</artifactId>
-				<version>2.12.1</version>
+				<version>2.16</version>
 				<executions>
 					<execution>
 						<id>validate</id>

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/tools/maven/checkstyle.xml
----------------------------------------------------------------------
diff --git a/tools/maven/checkstyle.xml b/tools/maven/checkstyle.xml
index a2864a0..ecedb19 100644
--- a/tools/maven/checkstyle.xml
+++ b/tools/maven/checkstyle.xml
@@ -52,7 +52,7 @@ under the License.
 		<module name="Regexp">
 			<property name="format" value="org\.apache\.commons\.lang3\.Validate"/>
 			<property name="illegalPattern" value="true"/>
-			<property name="message" value="Use Guava Checks instead of Commons Validate. Please refer coding guidelines."/>
+			<property name="message" value="Use Guava Checks instead of Commons Validate. Please refer to the coding guidelines."/>
 		</module>
 		<module name="NeedBraces">
 			<property name="tokens" value="LITERAL_IF, LITERAL_ELSE"/>


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
index 529d3d1..f11b933 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/TaskManager.scala
@@ -37,9 +37,10 @@ import com.fasterxml.jackson.databind.ObjectMapper
 import grizzled.slf4j.Logger
 
 import org.apache.flink.configuration._
-
+import org.apache.flink.core.memory.{HybridMemorySegment, HeapMemorySegment, MemorySegmentFactory, MemoryType}
 import org.apache.flink.runtime.accumulators.AccumulatorSnapshot
 import org.apache.flink.runtime.leaderretrieval.{LeaderRetrievalListener, LeaderRetrievalService}
+import org.apache.flink.runtime.memory.MemoryManager.HeapMemoryPool
 import org.apache.flink.runtime.messages.TaskMessages._
 import org.apache.flink.runtime.messages.checkpoint.{NotifyCheckpointComplete, TriggerCheckpoint, AbstractCheckpointMessage}
 import org.apache.flink.runtime.{FlinkActor, LeaderSessionMessageFilter, LogMessages, StreamingMode}
@@ -50,14 +51,13 @@ import org.apache.flink.runtime.deployment.{InputChannelDeploymentDescriptor, Ta
 import org.apache.flink.runtime.execution.librarycache.{BlobLibraryCacheManager, FallbackLibraryCacheManager, LibraryCacheManager}
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID
 import org.apache.flink.runtime.filecache.FileCache
-import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription,
-InstanceConnectionInfo, InstanceID}
+import org.apache.flink.runtime.instance.{AkkaActorGateway, HardwareDescription, InstanceConnectionInfo, InstanceID}
 import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode
 import org.apache.flink.runtime.io.disk.iomanager.{IOManager, IOManagerAsync}
 import org.apache.flink.runtime.io.network.NetworkEnvironment
 import org.apache.flink.runtime.io.network.netty.NettyConfig
 import org.apache.flink.runtime.jobgraph.IntermediateDataSetID
-import org.apache.flink.runtime.memorymanager.{MemoryManager, DefaultMemoryManager}
+import org.apache.flink.runtime.memory.MemoryManager 
 import org.apache.flink.runtime.messages.Messages._
 import org.apache.flink.runtime.messages.RegistrationMessages._
 import org.apache.flink.runtime.messages.TaskManagerMessages._
@@ -136,7 +136,7 @@ class TaskManager(
   protected val askTimeout = new Timeout(config.timeout)
 
   /** The TaskManager's physical execution resources */
-  protected val resources = HardwareDescription.extractFromSystem(memoryManager.getMemorySize)
+  protected val resources = HardwareDescription.extractFromSystem(memoryManager.getMemorySize())
 
   /** Registry of all tasks currently executed by this TaskManager */
   protected val runningTasks = new java.util.HashMap[ExecutionAttemptID, Task]()
@@ -1548,7 +1548,8 @@ object TaskManager {
 
     val (taskManagerConfig : TaskManagerConfiguration,      
       netConfig: NetworkEnvironmentConfiguration,
-      connectionInfo: InstanceConnectionInfo
+      connectionInfo: InstanceConnectionInfo,
+      memType: MemoryType
     ) = parseTaskManagerConfiguration(
       configuration,
       taskManagerHostname,
@@ -1577,7 +1578,7 @@ object TaskManager {
       LOG.info(s"Using $configuredMemory MB for Flink managed memory.")
       configuredMemory << 20 // megabytes to bytes
     }
-    else {
+    else if (memType == MemoryType.HEAP) {
       val fraction = configuration.getFloat(
         ConfigConstants.TASK_MANAGER_MEMORY_FRACTION_KEY,
         ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION)
@@ -1589,7 +1590,24 @@ object TaskManager {
         fraction).toLong
 
       LOG.info(s"Using $fraction of the currently free heap space for Flink managed " +
-        s"memory (${relativeMemSize >> 20} MB).")
+        s" heap memory (${relativeMemSize >> 20} MB).")
+
+      relativeMemSize
+    }
+    else {
+      val ratio = configuration.getFloat(
+        ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_RATIO_KEY,
+        ConfigConstants.DEFAULT_MEMORY_MANAGER_MEMORY_OFF_HEAP_RATIO)
+      
+      checkConfigParameter(ratio > 0.0f,
+        ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_RATIO_KEY,
+        "MemoryManager ratio (off-heap memory / heap size) must be larger than zero")
+      
+      val maxHeapSize = EnvironmentInformation.getMaxJvmHeapMemory()
+      val relativeMemSize = (maxHeapSize * ratio).toLong
+
+      LOG.info(s"Using $ratio time the heap size (${maxHeapSize} bytes) for Flink " +
+        s"managed off-heap memory (${relativeMemSize >> 20} MB).")
 
       relativeMemSize
     }
@@ -1598,16 +1616,27 @@ object TaskManager {
 
     // now start the memory manager
     val memoryManager = try {
-      new DefaultMemoryManager(
+      new MemoryManager(
         memorySize,
         taskManagerConfig.numberOfSlots,
         netConfig.networkBufferSize,
+        memType,
         preAllocateMemory)
     }
     catch {
-      case e: OutOfMemoryError => throw new Exception(
-        "OutOfMemory error (" + e.getMessage + ") while allocating the TaskManager memory (" +
-          memorySize + " bytes).", e)
+      case e: OutOfMemoryError => 
+        memType match {
+          case MemoryType.HEAP =>
+            throw new Exception(s"OutOfMemory error (${e.getMessage()})" + 
+              s" while allocating the TaskManager heap memory (${memorySize} bytes).", e)
+            
+          case MemoryType.OFF_HEAP =>
+            throw new Exception(s"OutOfMemory error (${e.getMessage()})" +
+              s" while allocating the TaskManager off-heap memory (${memorySize} bytes). " +
+              s"Try increasing the maximum direct memory (-XX:MaxDirectMemorySize)", e)
+            
+          case _ => throw e
+        }
     }
 
     // start the I/O manager last, it will create some temp directories.
@@ -1692,7 +1721,8 @@ object TaskManager {
       localTaskManagerCommunication: Boolean)
     : (TaskManagerConfiguration,
      NetworkEnvironmentConfiguration,
-     InstanceConnectionInfo) = {
+     InstanceConnectionInfo,
+     MemoryType) = {
 
     // ------- read values from the config and check them ---------
     //                      (a lot of them)
@@ -1738,9 +1768,9 @@ object TaskManager {
     val pageSize: Int =
       if (pageSizeNew != -1) {
         // new page size has been configured
-        checkConfigParameter(pageSizeNew >= DefaultMemoryManager.MIN_PAGE_SIZE, pageSizeNew,
+        checkConfigParameter(pageSizeNew >= MemoryManager.MIN_PAGE_SIZE, pageSizeNew,
           ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
-          "Minimum memory segment size is " + DefaultMemoryManager.MIN_PAGE_SIZE)
+          "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE)
 
         checkConfigParameter(MathUtils.isPowerOf2(pageSizeNew), pageSizeNew,
           ConfigConstants.TASK_MANAGER_MEMORY_SEGMENT_SIZE_KEY,
@@ -1754,9 +1784,9 @@ object TaskManager {
       }
       else {
         // old page size has been configured
-        checkConfigParameter(pageSizeOld >= DefaultMemoryManager.MIN_PAGE_SIZE, pageSizeOld,
+        checkConfigParameter(pageSizeOld >= MemoryManager.MIN_PAGE_SIZE, pageSizeOld,
           ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
-          "Minimum buffer size is " + DefaultMemoryManager.MIN_PAGE_SIZE)
+          "Minimum buffer size is " + MemoryManager.MIN_PAGE_SIZE)
 
         checkConfigParameter(MathUtils.isPowerOf2(pageSizeOld), pageSizeOld,
           ConfigConstants.TASK_MANAGER_NETWORK_BUFFER_SIZE_KEY,
@@ -1765,6 +1795,35 @@ object TaskManager {
         pageSizeOld
       }
     
+    // check whether we use heap or off-heap memory
+    val memType: MemoryType = 
+      if (configuration.getBoolean(ConfigConstants.TASK_MANAGER_MEMORY_OFF_HEAP_KEY, false)) {
+        MemoryType.OFF_HEAP
+      } else {
+        MemoryType.HEAP
+      }
+    
+    // initialize the memory segment factory accordingly
+    memType match {
+      case MemoryType.HEAP =>
+        if (!MemorySegmentFactory.isInitialized()) {
+          MemorySegmentFactory.initializeFactory(HeapMemorySegment.FACTORY)
+        }
+        else if (MemorySegmentFactory.getFactory() != HeapMemorySegment.FACTORY) {
+          throw new Exception("Memory type is set to heap memory, but memory segment " +
+            "factory has been initialized for off-heap memory segments")
+        }
+
+      case MemoryType.OFF_HEAP =>
+        if (!MemorySegmentFactory.isInitialized()) {
+          MemorySegmentFactory.initializeFactory(HybridMemorySegment.FACTORY)
+        }
+        else if (MemorySegmentFactory.getFactory() != HybridMemorySegment.FACTORY) {
+          throw new Exception("Memory type is set to off-heap memory, but memory segment " +
+            "factory has been initialized for heap memory segments")
+        }
+    }
+    
     val tmpDirs = configuration.getString(
       ConfigConstants.TASK_MANAGER_TMP_DIR_KEY,
       ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH)
@@ -1783,7 +1842,8 @@ object TaskManager {
     }
 
     // Default spill I/O mode for intermediate results
-    val syncOrAsync = configuration.getString(ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
+    val syncOrAsync = configuration.getString(
+      ConfigConstants.TASK_MANAGER_NETWORK_DEFAULT_IO_MODE,
       ConfigConstants.DEFAULT_TASK_MANAGER_NETWORK_DEFAULT_IO_MODE)
 
     val ioMode : IOMode = if (syncOrAsync == "async") IOMode.ASYNC else IOMode.SYNC
@@ -1791,6 +1851,7 @@ object TaskManager {
     val networkConfig = NetworkEnvironmentConfiguration(
       numNetworkBuffers,
       pageSize,
+      memType,
       ioMode,
       nettyConfig)
 
@@ -1834,7 +1895,7 @@ object TaskManager {
       slots,
       configuration)
 
-    (taskManagerConfig, networkConfig, connectionInfo)
+    (taskManagerConfig, networkConfig, connectionInfo, memType)
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
index f2a5e2d..a44916a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/ChannelViewsTest.java
@@ -22,6 +22,7 @@ package org.apache.flink.runtime.io.disk;
 import java.io.EOFException;
 import java.util.List;
 
+import org.apache.flink.core.memory.MemoryType;
 import org.junit.Assert;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
@@ -32,8 +33,7 @@ import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.TestData;
 import org.apache.flink.runtime.operators.testutils.TestData.Key;
@@ -78,7 +78,7 @@ public class ChannelViewsTest
 
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, true);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
index dcc1e5f..5deb50e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.runtime.io.disk;
 
 import static org.junit.Assert.*;
 
+import org.apache.flink.core.memory.MemoryType;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -29,8 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.PairGenerator;
 import org.apache.flink.runtime.operators.testutils.PairGenerator.KeyMode;
@@ -66,7 +66,8 @@ public class FileChannelStreamsITCase {
 
 	@Before
 	public void beforeTest() {
-		memManager = new DefaultMemoryManager(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE, 1, MEMORY_PAGE_SIZE, true);
+		memManager = new MemoryManager(NUM_MEMORY_SEGMENTS * MEMORY_PAGE_SIZE, 1,
+				MEMORY_PAGE_SIZE, MemoryType.HEAP, true);
 		ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
index f9b8b38..1c2b3de 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/FileChannelStreamsTest.java
@@ -26,13 +26,13 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.types.StringValue;
 import org.junit.Test;
@@ -44,7 +44,7 @@ public class FileChannelStreamsTest {
 	public void testCloseAndDeleteOutputView() {
 		final IOManager ioManager = new IOManagerAsync();
 		try {
-			MemoryManager memMan = new DefaultMemoryManager(4 * 16*1024, 1, 16*1024, true);
+			MemoryManager memMan = new MemoryManager(4 * 16*1024, 1, 16*1024, MemoryType.HEAP, true);
 			List<MemorySegment> memory = new ArrayList<MemorySegment>();
 			memMan.allocatePages(new DummyInvokable(), memory, 4);
 			
@@ -78,7 +78,7 @@ public class FileChannelStreamsTest {
 	public void testCloseAndDeleteInputView() {
 		final IOManager ioManager = new IOManagerAsync();
 		try {
-			MemoryManager memMan = new DefaultMemoryManager(4 * 16*1024, 1, 16*1024, true);
+			MemoryManager memMan = new MemoryManager(4 * 16*1024, 1, 16*1024, MemoryType.HEAP, true);
 			List<MemorySegment> memory = new ArrayList<MemorySegment>();
 			memMan.allocatePages(new DummyInvokable(), memory, 4);
 			

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
index c071bef..4c6a2b3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputViewTest.java
@@ -25,12 +25,12 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.junit.Test;
 
@@ -45,7 +45,7 @@ public class SeekableFileChannelInputViewTest {
 		// integers across 7.x pages (7 pages = 114.688 bytes, 8 pages = 131.072 bytes)
 		
 		try {
-			MemoryManager memMan = new DefaultMemoryManager(4 * PAGE_SIZE, 1, PAGE_SIZE, true);
+			MemoryManager memMan = new MemoryManager(4 * PAGE_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 			List<MemorySegment> memory = new ArrayList<MemorySegment>();
 			memMan.allocatePages(new DummyInvokable(), memory, 4);
 			
@@ -71,7 +71,7 @@ public class SeekableFileChannelInputViewTest {
 			try {
 				in.readInt();
 				fail("should throw EOF exception");
-			} catch (EOFException e) {}
+			} catch (EOFException ignored) {}
 			
 			// seek to the middle of the 3rd page
 			int i = 2 * PAGE_SIZE + PAGE_SIZE / 4;
@@ -82,7 +82,7 @@ public class SeekableFileChannelInputViewTest {
 			try {
 				in.readInt();
 				fail("should throw EOF exception");
-			} catch (EOFException e) {}
+			} catch (EOFException ignored) {}
 			
 			// seek to the end
 			i = 120000 - 4;
@@ -93,7 +93,7 @@ public class SeekableFileChannelInputViewTest {
 			try {
 				in.readInt();
 				fail("should throw EOF exception");
-			} catch (EOFException e) {}
+			} catch (EOFException ignored) {}
 			
 			// seek to the beginning
 			i = 0;
@@ -104,7 +104,7 @@ public class SeekableFileChannelInputViewTest {
 			try {
 				in.readInt();
 				fail("should throw EOF exception");
-			} catch (EOFException e) {}
+			} catch (EOFException ignored) {}
 			
 			// seek to after a page
 			i = PAGE_SIZE;
@@ -115,7 +115,7 @@ public class SeekableFileChannelInputViewTest {
 			try {
 				in.readInt();
 				fail("should throw EOF exception");
-			} catch (EOFException e) {}
+			} catch (EOFException ignored) {}
 			
 			// seek to after a page
 			i = 3 * PAGE_SIZE;
@@ -126,7 +126,7 @@ public class SeekableFileChannelInputViewTest {
 			try {
 				in.readInt();
 				fail("should throw EOF exception");
-			} catch (EOFException e) {}
+			} catch (EOFException ignored) {}
 			
 			// seek to the end
 			i = NUM_RECORDS;
@@ -134,17 +134,17 @@ public class SeekableFileChannelInputViewTest {
 			try {
 				in.readInt();
 				fail("should throw EOF exception");
-			} catch (EOFException e) {}
+			} catch (EOFException ignored) {}
 			
 			// seek out of bounds
 			try {
 				in.seek(-10);
 				fail("should throw an exception");
-			} catch (IllegalArgumentException e) {}
+			} catch (IllegalArgumentException ignored) {}
 			try {
 				in.seek(NUM_RECORDS + 1);
 				fail("should throw an exception");
-			} catch (IllegalArgumentException e) {}
+			} catch (IllegalArgumentException ignored) {}
 		}
 		catch (Exception e) {
 			e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
index 6a9a20a..0b1e0c3 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/SpillingBufferTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.disk;
 
 import org.apache.flink.core.memory.DataInputView;
@@ -24,9 +23,8 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.TestData;
 import org.apache.flink.runtime.operators.testutils.TestData.Generator.KeyMode;
@@ -34,6 +32,7 @@ import org.apache.flink.runtime.operators.testutils.TestData.Generator.ValueMode
 import org.apache.flink.runtime.operators.testutils.TestData.Key;
 import org.apache.flink.runtime.operators.testutils.TestData.Value;
 import org.apache.flink.types.Record;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -42,7 +41,6 @@ import org.junit.Test;
 import java.io.EOFException;
 import java.util.ArrayList;
 
-
 public class SpillingBufferTest {
 	
 	private static final long SEED = 649180756312423613L;
@@ -69,7 +67,7 @@ public class SpillingBufferTest {
 
 	@Before
 	public void beforeTest() {
-		memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		ioManager = new IOManagerAsync();
 	}
 
@@ -91,8 +89,7 @@ public class SpillingBufferTest {
 	// --------------------------------------------------------------------------------------------
 	
 	@Test
-	public void testWriteReadInMemory() throws Exception
-	{
+	public void testWriteReadInMemory() throws Exception {
 		final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
 		
 		// create the writer output view
@@ -150,9 +147,9 @@ public class SpillingBufferTest {
 	}
 	
 	@Test
-	public void testWriteReadTooMuchInMemory() throws Exception
-	{
-		final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+	public void testWriteReadTooMuchInMemory() throws Exception {
+		final TestData.Generator generator = new TestData.Generator(
+				SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
 		
 		// create the writer output view
 		final ArrayList<MemorySegment> memory = new ArrayList<MemorySegment>(NUM_MEMORY_SEGMENTS);
@@ -217,9 +214,9 @@ public class SpillingBufferTest {
 	// --------------------------------------------------------------------------------------------
 	
 	@Test
-	public void testWriteReadExternal() throws Exception
-	{
-		final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+	public void testWriteReadExternal() throws Exception {
+		final TestData.Generator generator = new TestData.Generator(
+				SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
 		
 		// create the writer output view
 		final ArrayList<MemorySegment> memory = new ArrayList<MemorySegment>(NUM_MEMORY_SEGMENTS);
@@ -276,9 +273,9 @@ public class SpillingBufferTest {
 	}
 
 	@Test
-	public void testWriteReadTooMuchExternal() throws Exception
-	{
-		final TestData.Generator generator = new TestData.Generator(SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
+	public void testWriteReadTooMuchExternal() throws Exception {
+		final TestData.Generator generator = new TestData.Generator(
+				SEED, KEY_MAX, VALUE_LENGTH, KeyMode.RANDOM, ValueMode.RANDOM_LENGTH);
 		
 		// create the writer output view
 		final ArrayList<MemorySegment> memory = new ArrayList<MemorySegment>(NUM_MEMORY_SEGMENTS);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
index 49e93c6..a471e66 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/AsynchronousFileIOChannelTest.java
@@ -18,7 +18,9 @@
 
 package org.apache.flink.runtime.io.disk.iomanager;
 
+import org.apache.flink.core.memory.HeapMemorySegment;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.util.TestNotificationListener;
 import org.junit.Test;
@@ -274,7 +276,7 @@ public class AsynchronousFileIOChannelTest {
 		try {
 
 			final int NUM_BLOCKS = 100;
-			final MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
+			final MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024);
 
 			final AtomicInteger callbackCounter = new AtomicInteger();
 			final AtomicBoolean exceptionOccurred = new AtomicBoolean();
@@ -336,7 +338,7 @@ public class AsynchronousFileIOChannelTest {
 
 	private void testExceptionForwardsToClose(IOManagerAsync ioMan, final int numBlocks, final int failingBlock) {
 		try {
-			MemorySegment seg = new MemorySegment(new byte[32 * 1024]);
+			MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024);
 			FileIOChannel.ID channelId = ioMan.createChannel();
 
 			BlockChannelWriterWithCallback<MemorySegment> writer = new AsynchronousBlockWriterWithCallback(channelId,
@@ -371,7 +373,7 @@ public class AsynchronousFileIOChannelTest {
 			finally {
 				try {
 					writer.closeAndDelete();
-				} catch (Throwable t) {}
+				} catch (Throwable ignored) {}
 			}
 		}
 		catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
index 294a6e6..c1bd465 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterFileSegmentReaderTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 import org.apache.flink.runtime.testutils.DiscardingRecycler;
@@ -146,7 +147,9 @@ public class BufferFileWriterFileSegmentReaderTest {
 
 			fileSegment.getFileChannel().read(buffer, fileSegment.getPosition());
 
-			currentNumber = verifyBufferFilledWithAscendingNumbers(new Buffer(new MemorySegment(buffer.array()), BUFFER_RECYCLER), currentNumber, fileSegment.getLength());
+			currentNumber = verifyBufferFilledWithAscendingNumbers(
+					new Buffer(MemorySegmentFactory.wrap(buffer.array()), BUFFER_RECYCLER), 
+					currentNumber, fileSegment.getLength());
 		}
 
 		reader.close();
@@ -169,7 +172,7 @@ public class BufferFileWriterFileSegmentReaderTest {
 	}
 
 	private Buffer createBuffer() {
-		return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER);
+		return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE), BUFFER_RECYCLER);
 	}
 
 	public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
index b0c702a..24d2864 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/BufferFileWriterReaderTest.java
@@ -19,9 +19,11 @@
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 import org.apache.flink.runtime.testutils.DiscardingRecycler;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -194,7 +196,7 @@ public class BufferFileWriterReaderTest {
 	}
 
 	private Buffer createBuffer() {
-		return new Buffer(new MemorySegment(new byte[BUFFER_SIZE]), BUFFER_RECYCLER);
+		return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(BUFFER_SIZE), BUFFER_RECYCLER);
 	}
 
 	public static int fillBufferWithAscendingNumbers(Buffer buffer, int currentNumber) {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
index 435588f..4656d56 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerAsyncTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.disk.iomanager;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -62,7 +63,7 @@ public class IOManagerAsyncTest {
 			final FileIOChannel.ID channelID = this.ioManager.createChannel();
 			final BlockChannelWriter<MemorySegment> writer = this.ioManager.createBlockChannelWriter(channelID);
 			
-			MemorySegment memSeg = new MemorySegment(new byte[32 * 1024]);
+			MemorySegment memSeg = MemorySegmentFactory.allocateUnpooledSegment(32 * 1024);
 			
 			for (int i = 0; i < NUM_IOS; i++) {
 				for (int pos = 0; pos < memSeg.size(); pos += 4) {
@@ -103,7 +104,7 @@ public class IOManagerAsyncTest {
 		try {
 			final List<MemorySegment> memSegs = new ArrayList<MemorySegment>();
 			for (int i = 0; i < NUM_SEGS; i++) {
-				memSegs.add(new MemorySegment(new byte[32 * 1024]));
+				memSegs.add(MemorySegmentFactory.allocateUnpooledSegment(32 * 1024));
 			}
 			
 			final FileIOChannel.ID channelID = this.ioManager.createChannel();

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
index 52908d3..6c25117 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerITCase.java
@@ -35,7 +35,7 @@ import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 
 /**
  * Integration test case for the I/O manager.
@@ -54,11 +54,11 @@ public class IOManagerITCase {
 
 	private IOManager ioManager;
 
-	private DefaultMemoryManager memoryManager;
+	private MemoryManager memoryManager;
 
 	@Before
 	public void beforeTest() {
-		memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		ioManager = new IOManagerAsync();
 	}
 
@@ -209,7 +209,7 @@ public class IOManagerITCase {
 		}
 	}
 	
-	private static final int skewedSample(Random rnd, int max) {
+	private static int skewedSample(Random rnd, int max) {
 		double uniform = rnd.nextDouble();
 		double var = Math.pow(uniform, 8.0);
 		double pareto = 0.2 / var;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
index 3bdc9bd..fd02623 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/disk/iomanager/IOManagerPerformanceBenchmark.java
@@ -42,7 +42,7 @@ import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 
 import org.junit.After;
 import org.junit.Before;
@@ -66,14 +66,14 @@ public class IOManagerPerformanceBenchmark {
 	
 	private static final AbstractInvokable memoryOwner = new DummyInvokable();
 	
-	private DefaultMemoryManager memManager;
+	private MemoryManager memManager;
 	
 	private IOManager ioManager;
 	
 	
 	@Before
 	public void startup() {
-		memManager = new DefaultMemoryManager(MEMORY_SIZE,1);
+		memManager = new MemoryManager(MEMORY_SIZE, 1);
 		ioManager = new IOManagerAsync();
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
index d657ebf..420199c 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.io.network;
 import static org.junit.Assert.*;
 
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.instance.DummyActorGateway;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
@@ -55,7 +56,8 @@ public class NetworkEnvironmentTest {
 		try {
 			NettyConfig nettyConf = new NettyConfig(InetAddress.getLocalHost(), port, BUFFER_SIZE, new Configuration());
 			NetworkEnvironmentConfiguration config = new NetworkEnvironmentConfiguration(
-					NUM_BUFFERS, BUFFER_SIZE, IOManager.IOMode.SYNC, new Some<NettyConfig>(nettyConf),
+					NUM_BUFFERS, BUFFER_SIZE, MemoryType.HEAP,
+					IOManager.IOMode.SYNC, new Some<NettyConfig>(nettyConf),
 					new Tuple2<Integer, Integer>(0, 0));
 
 			NetworkEnvironment env = new NetworkEnvironment(

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
index e7d0524..8455402 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/PagedViewsTest.java
@@ -16,20 +16,23 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.io.network.api.serialization;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestType;
 import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestTypeFactory;
 import org.apache.flink.runtime.io.network.api.serialization.types.Util;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
+
 import org.junit.Test;
 
 import java.io.EOFException;
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -375,7 +378,7 @@ public class PagedViewsTest {
 		private final int segmentSize;
 
 		private TestOutputView(int segmentSize) {
-			super(new MemorySegment(new byte[segmentSize]), segmentSize, 0);
+			super(MemorySegmentFactory.allocateUnpooledSegment(segmentSize), segmentSize, 0);
 
 			this.segmentSize = segmentSize;
 		}
@@ -383,7 +386,7 @@ public class PagedViewsTest {
 		@Override
 		protected MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException {
 			segments.add(new SegmentWithPosition(current, positionInCurrent));
-			return new MemorySegment(new byte[segmentSize]);
+			return MemorySegmentFactory.allocateUnpooledSegment(segmentSize);
 		}
 
 		public void close() {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
index cd6d580..819a94f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializationTest.java
@@ -19,11 +19,13 @@
 package org.apache.flink.runtime.io.network.api.serialization;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestType;
 import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestTypeFactory;
 import org.apache.flink.runtime.io.network.api.serialization.types.Util;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
+
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -125,7 +127,7 @@ public class SpanningRecordSerializationTest {
 	{
 		final int SERIALIZATION_OVERHEAD = 4; // length encoding
 
-		final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), mock(BufferRecycler.class));
+		final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(segmentSize), mock(BufferRecycler.class));
 
 		final ArrayDeque<SerializationTestType> serializedRecords = new ArrayDeque<SerializationTestType>();
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
index 50d3639..b7bcb3e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/serialization/SpanningRecordSerializerTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.io.network.api.serialization;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestType;
 import org.apache.flink.runtime.io.network.api.serialization.types.SerializationTestTypeFactory;
 import org.apache.flink.runtime.io.network.api.serialization.types.Util;
@@ -41,7 +42,7 @@ public class SpanningRecordSerializerTest {
 		final int SEGMENT_SIZE = 16;
 
 		final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
-		final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+		final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
 		final SerializationTestType randomIntRecord = Util.randomRecord(SerializationTestTypeFactory.INT);
 
 		Assert.assertFalse(serializer.hasData());
@@ -75,7 +76,7 @@ public class SpanningRecordSerializerTest {
 		final int SEGMENT_SIZE = 11;
 
 		final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
-		final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+		final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
 
 		try {
 			Assert.assertEquals(RecordSerializer.SerializationResult.FULL_RECORD, serializer.setNextBuffer(buffer));
@@ -201,7 +202,7 @@ public class SpanningRecordSerializerTest {
 		final int SERIALIZATION_OVERHEAD = 4; // length encoding
 
 		final SpanningRecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
-		final Buffer buffer = new Buffer(new MemorySegment(new byte[segmentSize]), mock(BufferRecycler.class));
+		final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(segmentSize), mock(BufferRecycler.class));
 
 		// -------------------------------------------------------------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
index 9e10582..f8cd28f 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/api/writer/RecordWriterTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.io.network.api.writer;
 
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferPool;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
@@ -25,9 +26,11 @@ import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool;
 import org.apache.flink.runtime.io.network.util.TestBufferFactory;
 import org.apache.flink.runtime.io.network.util.TestTaskEvent;
 import org.apache.flink.types.IntValue;
+
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
+
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 import org.powermock.core.classloader.annotations.PrepareForTest;
@@ -41,14 +44,12 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 
 import static com.google.common.base.Preconditions.checkNotNull;
+
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
-import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.spy;
@@ -158,7 +159,7 @@ public class RecordWriterTest {
 		BufferPool bufferPool = null;
 
 		try {
-			buffers = new NetworkBufferPool(1, 1024);
+			buffers = new NetworkBufferPool(1, 1024, MemoryType.HEAP);
 			bufferPool = spy(buffers.createBufferPool(1, true));
 
 			ResultPartitionWriter partitionWriter = mock(ResultPartitionWriter.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
index 28862e8..0ac84dc 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferPoolFactoryTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.io.network.buffer;
 
+import org.apache.flink.core.memory.MemoryType;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -40,7 +41,7 @@ public class BufferPoolFactoryTest {
 
 	@Before
 	public void setupNetworkBufferPool() {
-		networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize);
+		networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize, MemoryType.HEAP);
 	}
 
 	@After

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
index 734dcfb..fd11d02 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/BufferTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.network.buffer;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.junit.Assert;
 import org.junit.Test;
 import org.mockito.Mockito;
@@ -32,7 +33,7 @@ public class BufferTest {
 
 	@Test
 	public void testSetGetSize() {
-		final MemorySegment segment = new MemorySegment(new byte[1024]);
+		final MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(1024);
 		final BufferRecycler recycler = Mockito.mock(BufferRecycler.class);
 
 		Buffer buffer = new Buffer(segment, recycler);
@@ -58,7 +59,7 @@ public class BufferTest {
 
 	@Test
 	public void testgetNioBufferThreadSafe() {
-		final MemorySegment segment = new MemorySegment(new byte[1024]);
+		final MemorySegment segment = MemorySegmentFactory.allocateUnpooledSegment(1024);
 		final BufferRecycler recycler = Mockito.mock(BufferRecycler.class);
 
 		Buffer buffer = new Buffer(segment, recycler);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
index e8e9ec8..93731e1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/LocalBufferPoolTest.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.network.buffer;
 
 import com.google.common.collect.Lists;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.util.event.EventListener;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -61,7 +62,7 @@ public class LocalBufferPoolTest {
 
 	@Before
 	public void setupLocalBufferPool() {
-		networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize);
+		networkBufferPool = new NetworkBufferPool(numBuffers, memorySegmentSize, MemoryType.HEAP);
 		localBufferPool = new LocalBufferPool(networkBufferPool, 1);
 
 		assertEquals(0, localBufferPool.getNumberOfAvailableMemorySegments());

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
index 6b22cd9..fd5c7a5 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPoolTest.java
@@ -18,6 +18,7 @@
 
 package org.apache.flink.runtime.io.network.buffer;
 
+import org.apache.flink.core.memory.MemoryType;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -35,7 +36,7 @@ public class NetworkBufferPoolTest {
 			final int bufferSize = 128;
 			final int numBuffers = 10;
 
-			NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize);
+			NetworkBufferPool globalPool = new NetworkBufferPool(numBuffers, bufferSize, MemoryType.HEAP);
 			assertEquals(bufferSize, globalPool.getMemorySegmentSize());
 			assertEquals(numBuffers, globalPool.getTotalNumberOfMemorySegments());
 			assertEquals(numBuffers, globalPool.getNumberOfAvailableMemorySegments());
@@ -70,7 +71,7 @@ public class NetworkBufferPoolTest {
 	@Test
 	public void testDestroyAll() {
 		try {
-			NetworkBufferPool globalPool = new NetworkBufferPool(10, 128);
+			NetworkBufferPool globalPool = new NetworkBufferPool(10, 128, MemoryType.HEAP);
 
 			BufferPool fixedPool = globalPool.createBufferPool(2, true);
 			BufferPool nonFixedPool = globalPool.createBufferPool(5, false);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
index 60241e3..f514cbd 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyMessageSerializationTest.java
@@ -20,7 +20,7 @@ package org.apache.flink.runtime.io.network.netty;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.channel.embedded.EmbeddedChannel;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.event.task.IntegerTaskEvent;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
@@ -53,7 +53,7 @@ public class NettyMessageSerializationTest {
 	@Test
 	public void testEncodeDecode() {
 		{
-			Buffer buffer = spy(new Buffer(new MemorySegment(new byte[1024]), mock(BufferRecycler.class)));
+			Buffer buffer = spy(new Buffer(MemorySegmentFactory.allocateUnpooledSegment(1024), mock(BufferRecycler.class)));
 			ByteBuffer nioBuffer = buffer.getNioBuffer();
 
 			for (int i = 0; i < 1024; i += 4) {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
index 65780b4..cfbe99e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/IteratorWrappingTestSingleInputGate.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.core.io.IOReadableWritable;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordSerializer;
@@ -68,7 +68,7 @@ public class IteratorWrappingTestSingleInputGate<T extends IOReadableWritable> e
 			@Override
 			public Buffer answer(InvocationOnMock invocationOnMock) throws Throwable {
 				if (inputIterator.next(reuse) != null) {
-					final Buffer buffer = new Buffer(new MemorySegment(new byte[bufferSize]), mock(BufferRecycler.class));
+					final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), mock(BufferRecycler.class));
 					serializer.setNextBuffer(buffer);
 					serializer.addRecord(reuse);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
index cd56318..ea40a55 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/LocalInputChannelTest.java
@@ -21,6 +21,7 @@ package org.apache.flink.runtime.io.network.partition.consumer;
 import com.google.common.collect.Lists;
 
 import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.execution.CancelTaskException;
 import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
@@ -94,7 +95,7 @@ public class LocalInputChannelTest {
 
 		final NetworkBufferPool networkBuffers = new NetworkBufferPool(
 				(parallelism * producerBufferPoolSize) + (parallelism * parallelism),
-				TestBufferFactory.BUFFER_SIZE);
+				TestBufferFactory.BUFFER_SIZE, MemoryType.HEAP);
 
 		final ResultPartitionConsumableNotifier partitionConsumableNotifier =
 				mock(ResultPartitionConsumableNotifier.class);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
index 82cc730..f4c37f4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
 import org.apache.flink.runtime.deployment.ResultPartitionLocation;
 import org.apache.flink.runtime.event.TaskEvent;
@@ -100,7 +100,8 @@ public class SingleInputGateTest {
 		when(taskEventDispatcher.publish(any(ResultPartitionID.class), any(TaskEvent.class))).thenReturn(true);
 
 		final ResultSubpartitionView iterator = mock(ResultSubpartitionView.class);
-		when(iterator.getNextBuffer()).thenReturn(new Buffer(new MemorySegment(new byte[1024]), mock(BufferRecycler.class)));
+		when(iterator.getNextBuffer()).thenReturn(
+				new Buffer(MemorySegmentFactory.allocateUnpooledSegment(1024), mock(BufferRecycler.class)));
 
 		final ResultPartitionManager partitionManager = mock(ResultPartitionManager.class);
 		when(partitionManager.createSubpartitionView(any(ResultPartitionID.class), anyInt(), any(BufferProvider.class))).thenReturn(iterator);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
index d9e3562..d628596 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/serialization/LargeRecordsTest.java
@@ -18,7 +18,7 @@
 
 package org.apache.flink.runtime.io.network.serialization;
 
-import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.AdaptiveSpanningRecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
@@ -52,7 +52,7 @@ public class LargeRecordsTest {
 			final RecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
 			final RecordDeserializer<SerializationTestType> deserializer = new AdaptiveSpanningRecordDeserializer<SerializationTestType>();
 
-			final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+			final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
 
 			List<SerializationTestType> originalRecords = new ArrayList<SerializationTestType>();
 			List<SerializationTestType> deserializedRecords = new ArrayList<SerializationTestType>();
@@ -149,7 +149,7 @@ public class LargeRecordsTest {
 			final RecordSerializer<SerializationTestType> serializer = new SpanningRecordSerializer<SerializationTestType>();
 			final RecordDeserializer<SerializationTestType> deserializer = new SpillingAdaptiveSpanningRecordDeserializer<SerializationTestType>();
 
-			final Buffer buffer = new Buffer(new MemorySegment(new byte[SEGMENT_SIZE]), mock(BufferRecycler.class));
+			final Buffer buffer = new Buffer(MemorySegmentFactory.allocateUnpooledSegment(SEGMENT_SIZE), mock(BufferRecycler.class));
 
 			List<SerializationTestType> originalRecords = new ArrayList<SerializationTestType>();
 			List<SerializationTestType> deserializedRecords = new ArrayList<SerializationTestType>();

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
index cdba545..4b3b465 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/util/TestBufferFactory.java
@@ -18,7 +18,9 @@
 
 package org.apache.flink.runtime.io.network.util;
 
+import org.apache.flink.core.memory.HeapMemorySegment;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 import org.apache.flink.runtime.testutils.DiscardingRecycler;
@@ -59,7 +61,7 @@ public class TestBufferFactory {
 	public Buffer create() {
 		numberOfCreatedBuffers.incrementAndGet();
 
-		return new Buffer(new MemorySegment(new byte[bufferSize]), bufferRecycler);
+		return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), bufferRecycler);
 	}
 
 	public Buffer createFrom(MemorySegment segment) {
@@ -85,7 +87,7 @@ public class TestBufferFactory {
 	public static Buffer createBuffer(int bufferSize) {
 		checkArgument(bufferSize > 0);
 
-		return new Buffer(new MemorySegment(new byte[bufferSize]), RECYCLER);
+		return new Buffer(MemorySegmentFactory.allocateUnpooledSegment(bufferSize), RECYCLER);
 	}
 
 	public static Buffer getMockBuffer() {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
index fc88207..15251e9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerLazyAllocationTest.java
@@ -19,9 +19,8 @@
 package org.apache.flink.runtime.memory;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 
 import org.junit.After;
@@ -48,14 +47,14 @@ public class MemoryManagerLazyAllocationTest {
 	
 	private static final int NUM_PAGES = MEMORY_SIZE / PAGE_SIZE;
 
-	private DefaultMemoryManager memoryManager;
+	private MemoryManager memoryManager;
 
 	private Random random;
 
 	
 	@Before
 	public void setUp() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, false);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, false);
 		this.random = new Random(RANDOM_SEED);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
index c0f32ca..a20a180 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerTest.java
@@ -23,9 +23,8 @@ import java.util.List;
 import java.util.Random;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
 
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.junit.Assert;
@@ -48,14 +47,14 @@ public class MemoryManagerTest {
 	
 	private static final int NUM_PAGES = MEMORY_SIZE / PAGE_SIZE;
 
-	private DefaultMemoryManager memoryManager;
+	private MemoryManager memoryManager;
 
 	private Random random;
 
 	
 	@Before
 	public void setUp() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, true);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 		this.random = new Random(RANDOM_SEED);
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java
new file mode 100644
index 0000000..fad1b0e
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSimpleTest.java
@@ -0,0 +1,576 @@
+/*
+ * 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.runtime.memory;
+
+import static org.junit.Assert.assertArrayEquals;
+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.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.operators.testutils.DummyInvokable;
+import org.apache.flink.core.memory.MemorySegment;
+
+import org.junit.Assert;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MemorySegmentSimpleTest {
+	
+	public static final long RANDOM_SEED = 643196033469871L;
+
+	public static final int MANAGED_MEMORY_SIZE = 1024 * 1024 * 16;
+
+	public static final int PAGE_SIZE = 1024 * 512;
+
+	private MemoryManager manager;
+
+	private MemorySegment segment;
+
+	private Random random;
+
+	@Before
+	public void setUp() throws Exception{
+		try {
+			this.manager = new MemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
+			this.segment = manager.allocatePages(new DummyInvokable(), 1).get(0);
+			this.random = new Random(RANDOM_SEED);
+		} catch (Exception e) {
+			e.printStackTrace();
+			fail("Test setup failed.");
+		}
+	}
+
+	@After
+	public void tearDown() {
+		this.manager.release(this.segment);
+		this.random = null;
+		this.segment = null;
+		
+		if (!this.manager.verifyEmpty()) {
+			Assert.fail("Not all memory has been properly released.");
+		}
+		this.manager = null;
+	}
+
+	@Test
+	public void bulkByteAccess() {
+
+		// test exceptions
+		{
+			byte[] bytes = new byte[PAGE_SIZE / 4];
+
+			try {
+				segment.put(3 * (PAGE_SIZE / 4) + 1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(7 * (PAGE_SIZE / 8) + 1, bytes, 0, bytes.length / 2);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior with default offset / length
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			byte[] src = new byte[PAGE_SIZE / 8];
+			for (int i = 0; i < 8; i++) {
+				random.nextBytes(src);
+				segment.put(i * (PAGE_SIZE / 8), src);
+			}
+
+			random.setSeed(seed);
+			byte[] expected = new byte[PAGE_SIZE / 8];
+			byte[] actual = new byte[PAGE_SIZE / 8];
+			for (int i = 0; i < 8; i++) {
+				random.nextBytes(expected);
+				segment.get(i * (PAGE_SIZE / 8), actual);
+
+				assertArrayEquals(expected, actual);
+			}
+		}
+
+		// test expected correct behavior with specific offset / length
+		{
+			byte[] expected = new byte[PAGE_SIZE];
+			random.nextBytes(expected);
+
+			for (int i = 0; i < 16; i++) {
+				segment.put(i * (PAGE_SIZE / 16), expected, i * (PAGE_SIZE / 16),
+					PAGE_SIZE / 16);
+			}
+
+			byte[] actual = new byte[PAGE_SIZE];
+			for (int i = 0; i < 16; i++) {
+				segment.get(i * (PAGE_SIZE / 16), actual, i * (PAGE_SIZE / 16),
+					PAGE_SIZE / 16);
+			}
+
+			assertArrayEquals(expected, actual);
+		}
+	}
+
+	@Test
+	public void byteAccess() {
+		// test exceptions
+		{
+			try {
+				segment.put(-1, (byte) 0);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(PAGE_SIZE, (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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i < PAGE_SIZE; i++) {
+				segment.put(i, (byte) random.nextInt());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i < PAGE_SIZE; i++) {
+				assertEquals((byte) random.nextInt(), segment.get(i));
+			}
+		}
+	}
+
+	@Test
+	public void booleanAccess() {
+		// test exceptions
+		{
+			try {
+				segment.putBoolean(-1, false);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putBoolean(PAGE_SIZE, false);
+				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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i < PAGE_SIZE; i++) {
+				segment.putBoolean(i, random.nextBoolean());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i < PAGE_SIZE; i++) {
+				assertEquals(random.nextBoolean(), segment.getBoolean(i));
+			}
+		}
+	}
+
+	@Test
+	public void charAccess() {
+		// test exceptions
+		{
+			try {
+				segment.putChar(-1, 'a');
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putChar(PAGE_SIZE, 'a');
+				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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+				segment.putChar(i, (char) ('a' + random.nextInt(26)));
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+				assertEquals((char) ('a' + random.nextInt(26)), segment.getChar(i));
+			}
+		}
+	}
+
+	@Test
+	public void doubleAccess() {
+		// test exceptions
+		{
+			try {
+				segment.putDouble(-1, 0.0);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putDouble(PAGE_SIZE, 0.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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+				segment.putDouble(i, random.nextDouble());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+				assertEquals(random.nextDouble(), segment.getDouble(i), 0.0);
+			}
+		}
+	}
+
+	// @Test
+	public void floatAccess() {
+		// test exceptions
+		{
+			try {
+				segment.putFloat(-1, 0.0f);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putFloat(PAGE_SIZE, 0.0f);
+				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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+				segment.putFloat(i, random.nextFloat());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+				assertEquals(random.nextFloat(), segment.getFloat(i), 0.0);
+			}
+		}
+	}
+
+	@Test
+	public void longAccess() {
+		// test exceptions
+		{
+			try {
+				segment.putLong(-1, 0L);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putLong(PAGE_SIZE, 0L);
+				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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+				segment.putLong(i, random.nextLong());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
+				assertEquals(random.nextLong(), segment.getLong(i));
+			}
+		}
+		
+		// test unaligned offsets
+		{
+			final long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
+				long value = random.nextLong();
+				segment.putLong(offset, value);
+			}
+			
+			random.setSeed(seed);
+			for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
+				long shouldValue = random.nextLong();
+				long isValue = segment.getLong(offset);
+				assertEquals(shouldValue, isValue);
+			}
+		}
+	}
+
+	@Test
+	public void intAccess() {
+		// test exceptions
+		{
+			try {
+				segment.putInt(-1, 0);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putInt(PAGE_SIZE, 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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+				segment.putInt(i, random.nextInt());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
+				assertEquals(random.nextInt(), segment.getInt(i));
+			}
+		}
+	}
+
+	@Test
+	public void shortAccess() {
+		// test exceptions
+		{
+			try {
+				segment.putShort(-1, (short) 0);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putShort(PAGE_SIZE, (short) 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(PAGE_SIZE);
+				fail("IndexOutOfBoundsException expected");
+			} catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+
+		// test expected correct behavior
+		{
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+				segment.putShort(i, (short) random.nextInt());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
+				assertEquals((short) random.nextInt(), segment.getShort(i));
+			}
+		}
+	}
+	
+	@Test
+	public void testByteBufferWrapping() {
+		try {
+			MemorySegment seg = MemorySegmentFactory.allocateUnpooledSegment(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());
+		}
+	}
+}


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java
deleted file mode 100644
index 0843cc2..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentSpeedBenchmark.java
+++ /dev/null
@@ -1,867 +0,0 @@
-/*
- * 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.runtime.memory;
-
-import java.util.Arrays;
-
-import org.apache.flink.runtime.memorymanager.CheckedMemorySegment;
-import org.apache.flink.runtime.memorymanager.SimpleMemorySegment;
-import org.apache.flink.runtime.memorymanager.UnsafeMemorySegment;
-
-/**
- *
- */
-public class MemorySegmentSpeedBenchmark {
-	
-	private static final long LONG_VALUE = 0x1234567890abcdefl;
-	
-	private static final int INT_VALUE = 0x12345678;
-	
-	private static final byte BYTE_VALUE = 0x56;
-	
-	@SuppressWarnings("unused")
-	private static long sideEffect = 0;
-	
-	
-	public static void main(String[] args) {
-		final int SMALL_SEGMENT_SIZE = 32 * 1024;
-		final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
-		
-		final int SMALL_SEGMENTS_ROUNDS = 50000;
-		final int LARGE_SEGMENT_ROUNDS = 10;
-		
-		final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
-		final byte[] smallSegment = new byte[SMALL_SEGMENT_SIZE];
-		
-		testPutLongs(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
-		testGetLongs(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
-		testPutLongs(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-		testGetLongs(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-		
-		testPutLongsBigEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
-		testGetLongsBigEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
-		testPutLongsBigEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-		testGetLongsBigEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-		
-		testPutLongsLittleEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
-		testGetLongsLittleEndian(smallSegment, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
-		testPutLongsLittleEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-		testGetLongsLittleEndian(largeSegment, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
-		
-		testPutInts(smallSegment, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
-		testGetInts(smallSegment, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
-		testPutInts(largeSegment, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
-		testGetInts(largeSegment, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
-		
-		testPutBytes(smallSegment, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
-		testGetBytes(smallSegment, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
-		testPutBytes(largeSegment, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
-		testGetBytes(largeSegment, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
-		
-		testPutByteArrays1024(smallSegment, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
-		testGetByteArrays1024(smallSegment, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
-		testPutByteArrays1024(largeSegment, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
-		testGetByteArrays1024(largeSegment, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                  LONGs
-	// --------------------------------------------------------------------------------------------
-	
-	private static final void testPutLongs(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timePutLongsChecked(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timePutLongsDirect(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timePutLongsUnsafe(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Writing %d x %d longs to %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static final void testGetLongs(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timeGetLongsChecked(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timeGetLongsDirect(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timeGetLongsUnsafe(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Reading %d x %d longs from %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static long timePutLongsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.putLong(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutLongsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.putLong(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutLongsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.putLong(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetLongsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += checked.getLong(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetLongsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += direct.getLong(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetLongsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += unsafe.getLong(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                  LONG BIG ENDIAN
-	// --------------------------------------------------------------------------------------------
-	
-	private static final void testPutLongsBigEndian(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timePutLongsCheckedBigEndian(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timePutLongsDirectBigEndian(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timePutLongsUnsafeBigEndian(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Writing %d x %d big endian longs to %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static final void testGetLongsBigEndian(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timeGetLongsCheckedBigEndian(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timeGetLongsDirectBigEndian(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timeGetLongsUnsafeBigEndian(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Reading %d x %d big endian longs from %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static long timePutLongsCheckedBigEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.putLongBigEndian(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutLongsDirectBigEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.putLongBigEndian(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutLongsUnsafeBigEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.putLongBigEndian(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetLongsCheckedBigEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += checked.getLongBigEndian(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetLongsDirectBigEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += direct.getLongBigEndian(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetLongsUnsafeBigEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += unsafe.getLongBigEndian(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                  LONG LITTLE ENDIAN
-	// --------------------------------------------------------------------------------------------
-	
-	private static final void testPutLongsLittleEndian(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timePutLongsCheckedLittleEndian(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timePutLongsDirectLittleEndian(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timePutLongsUnsafeLittleEndian(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Writing %d x %d little endian longs to %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static final void testGetLongsLittleEndian(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timeGetLongsCheckedLittleEndian(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timeGetLongsDirectLittleEndian(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timeGetLongsUnsafeLittleEndian(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Reading %d x %d little endian longs from %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static long timePutLongsCheckedLittleEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.putLongLittleEndian(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutLongsDirectLittleEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.putLongLittleEndian(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutLongsUnsafeLittleEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.putLongLittleEndian(offset, LONG_VALUE);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetLongsCheckedLittleEndian(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += checked.getLongLittleEndian(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetLongsDirectLittleEndian(final SimpleMemorySegment direct, final int num, final int rounds) {
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += direct.getLongLittleEndian(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetLongsUnsafeLittleEndian(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += unsafe.getLongLittleEndian(offset);
-				offset += 8;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                  INTs
-	// --------------------------------------------------------------------------------------------
-	
-	private static final void testPutInts(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timePutIntsChecked(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timePutIntsDirect(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timePutIntsUnsafe(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Writing %d x %d ints to %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static final void testGetInts(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timeGetIntsChecked(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timeGetIntsDirect(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timeGetIntsUnsafe(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Reading %d x %d ints from %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static long timePutIntsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.putInt(offset, INT_VALUE);
-				offset += 4;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutIntsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.putInt(offset, INT_VALUE);
-				offset += 4;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutIntsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.putInt(offset, INT_VALUE);
-				offset += 4;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetIntsChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
-		int l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += checked.getInt(offset);
-				offset += 4;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetIntsDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
-		int l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += direct.getInt(offset);
-				offset += 4;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	private static long timeGetIntsUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		int l = 0;
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				l += unsafe.getInt(offset);
-				offset += 4;
-			}
-		}
-		long end = System.nanoTime();
-		sideEffect += l;
-		return end - start;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                  BYTEs
-	// --------------------------------------------------------------------------------------------
-	
-	private static final void testPutBytes(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timePutBytesChecked(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timePutBytesDirect(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timePutBytesUnsafe(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Writing %d x %d bytes to %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static final void testGetBytes(byte[] segmentArray, int numValues, int rounds) {
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timeGetBytesChecked(checkedSegment, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timeGetBytesDirect(directSegment, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timeGetBytesUnsafe(unsafeSegment, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Reading %d x %d bytes from %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static long timePutBytesChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.put(offset, BYTE_VALUE);
-				offset++;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutBytesDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.put(offset, BYTE_VALUE);
-				offset++;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutBytesUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.put(offset, BYTE_VALUE);
-				offset++;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetBytesChecked(final CheckedMemorySegment checked, final int num, final int rounds) {
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.get(offset);
-				offset++;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetBytesDirect(final SimpleMemorySegment direct, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.get(offset);
-				offset++;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetBytesUnsafe(final UnsafeMemorySegment unsafe, final int num, final int rounds) {
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.get(offset);
-				offset++;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                                  BYTE ARRAYs
-	// --------------------------------------------------------------------------------------------
-	
-	private static final void testPutByteArrays1024(byte[] segmentArray, int numValues, int rounds) {
-		byte[] sourceArray = new byte[1024];
-		for (int i = 0; i < sourceArray.length; i++) {
-			sourceArray[i] = (byte) (i % Byte.MAX_VALUE);
-		}
-
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timePutByteArrayChecked(checkedSegment, sourceArray, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timePutByteArrayDirect(directSegment, sourceArray, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timePutByteArrayUnsafe(unsafeSegment, sourceArray, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Writing %d x %d byte[1024] to %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-	private static final void testGetByteArrays1024(byte[] segmentArray, int numValues, int rounds) {
-		byte[] targetArray = new byte[1024];
-
-		Arrays.fill(segmentArray, (byte) 0);
-		CheckedMemorySegment checkedSegment = new CheckedMemorySegment(segmentArray);
-		long elapsedChecked = timeGetByteArrayChecked(checkedSegment, targetArray, numValues, rounds);
-		checkedSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		SimpleMemorySegment directSegment = new SimpleMemorySegment(segmentArray);
-		long elapsedDirect = timeGetByteArrayDirect(directSegment, targetArray, numValues, rounds);
-		directSegment = null;
-		
-		Arrays.fill(segmentArray, (byte) 0);
-		UnsafeMemorySegment unsafeSegment = new UnsafeMemorySegment(segmentArray);
-		long elapsedUnsafe = timeGetByteArrayUnsafe(unsafeSegment, targetArray, numValues, rounds);
-		unsafeSegment = null;
-		
-		System.out.println(String.format("Reading %d x %d byte[1024] from %d segment: checked=%,d nsecs,  direct=%,d nsecs, unsafe=%,d nsecs.", rounds, numValues, segmentArray.length, elapsedChecked, elapsedDirect, elapsedUnsafe));
-	}
-	
-
-	private static long timePutByteArrayChecked(final CheckedMemorySegment checked, final byte[] source, final int num, final int rounds) {
-		final int len = source.length;
-		
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.put(offset, source, 0, len);
-				offset += len;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutByteArrayDirect(final SimpleMemorySegment direct, final byte[] source, final int num, final int rounds) {
-		final int len = source.length;
-		
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.put(offset, source, 0, len);
-				offset += len;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timePutByteArrayUnsafe(final UnsafeMemorySegment unsafe, final byte[] source, final int num, final int rounds) {
-		final int len = source.length;
-		
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.put(offset, source, 0, len);
-				offset += len;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetByteArrayChecked(final CheckedMemorySegment checked, final byte[] target, final int num, final int rounds) {
-		final int len = target.length;
-		
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				checked.get(offset, target, 0, len);
-				offset += len;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetByteArrayDirect(final SimpleMemorySegment direct, final byte[] target, final int num, final int rounds) {
-		final int len = target.length;
-		
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				direct.get(offset, target, 0, len);
-				offset += len;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-	
-	private static long timeGetByteArrayUnsafe(final UnsafeMemorySegment unsafe, final byte[] target, final int num, final int rounds) {
-		final int len = target.length;
-		
-		// checked segment
-		long start = System.nanoTime();
-		for (int round = 0; round < rounds; round++) {
-			int offset = 0;
-			for (int i = 0; i < num; i++) {
-				unsafe.get(offset, target, 0, len);
-				offset += len;
-			}
-		}
-		long end = System.nanoTime();
-		return end - start;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/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
deleted file mode 100644
index 990dffa..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemorySegmentTest.java
+++ /dev/null
@@ -1,575 +0,0 @@
-/*
- * 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.runtime.memory;
-
-import static org.junit.Assert.assertArrayEquals;
-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.apache.flink.runtime.operators.testutils.DummyInvokable;
-import org.junit.Assert;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class MemorySegmentTest {
-	
-	public static final long RANDOM_SEED = 643196033469871L;
-
-	public static final int MANAGED_MEMORY_SIZE = 1024 * 1024 * 16;
-
-	public static final int PAGE_SIZE = 1024 * 512;
-
-	private DefaultMemoryManager manager;
-
-	private MemorySegment segment;
-
-	private Random random;
-
-	@Before
-	public void setUp() throws Exception{
-		try {
-			this.manager = new DefaultMemoryManager(MANAGED_MEMORY_SIZE, 1, PAGE_SIZE, true);
-			this.segment = manager.allocatePages(new DummyInvokable(), 1).get(0);
-			this.random = new Random(RANDOM_SEED);
-		} catch (Exception e) {
-			e.printStackTrace();
-			fail("Test setup failed.");
-		}
-	}
-
-	@After
-	public void tearDown() {
-		this.manager.release(this.segment);
-		this.random = null;
-		this.segment = null;
-		
-		if (!this.manager.verifyEmpty()) {
-			Assert.fail("Not all memory has been properly released.");
-		}
-		this.manager = null;
-	}
-
-	@Test
-	public void bulkByteAccess() {
-
-		// test exceptions
-		{
-			byte[] bytes = new byte[PAGE_SIZE / 4];
-
-			try {
-				segment.put(3 * (PAGE_SIZE / 4) + 1, bytes);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.put(7 * (PAGE_SIZE / 8) + 1, bytes, 0, bytes.length / 2);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior with default offset / length
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			byte[] src = new byte[PAGE_SIZE / 8];
-			for (int i = 0; i < 8; i++) {
-				random.nextBytes(src);
-				segment.put(i * (PAGE_SIZE / 8), src);
-			}
-
-			random.setSeed(seed);
-			byte[] expected = new byte[PAGE_SIZE / 8];
-			byte[] actual = new byte[PAGE_SIZE / 8];
-			for (int i = 0; i < 8; i++) {
-				random.nextBytes(expected);
-				segment.get(i * (PAGE_SIZE / 8), actual);
-
-				assertArrayEquals(expected, actual);
-			}
-		}
-
-		// test expected correct behavior with specific offset / length
-		{
-			byte[] expected = new byte[PAGE_SIZE];
-			random.nextBytes(expected);
-
-			for (int i = 0; i < 16; i++) {
-				segment.put(i * (PAGE_SIZE / 16), expected, i * (PAGE_SIZE / 16),
-					PAGE_SIZE / 16);
-			}
-
-			byte[] actual = new byte[PAGE_SIZE];
-			for (int i = 0; i < 16; i++) {
-				segment.get(i * (PAGE_SIZE / 16), actual, i * (PAGE_SIZE / 16),
-					PAGE_SIZE / 16);
-			}
-
-			assertArrayEquals(expected, actual);
-		}
-	}
-
-	@Test
-	public void byteAccess() {
-		// test exceptions
-		{
-			try {
-				segment.put(-1, (byte) 0);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.put(PAGE_SIZE, (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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i < PAGE_SIZE; i++) {
-				segment.put(i, (byte) random.nextInt());
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i < PAGE_SIZE; i++) {
-				assertEquals((byte) random.nextInt(), segment.get(i));
-			}
-		}
-	}
-
-	@Test
-	public void booleanAccess() {
-		// test exceptions
-		{
-			try {
-				segment.putBoolean(-1, false);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.putBoolean(PAGE_SIZE, false);
-				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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i < PAGE_SIZE; i++) {
-				segment.putBoolean(i, random.nextBoolean());
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i < PAGE_SIZE; i++) {
-				assertEquals(random.nextBoolean(), segment.getBoolean(i));
-			}
-		}
-	}
-
-	@Test
-	public void charAccess() {
-		// test exceptions
-		{
-			try {
-				segment.putChar(-1, 'a');
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.putChar(PAGE_SIZE, 'a');
-				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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
-				segment.putChar(i, (char) ('a' + random.nextInt(26)));
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
-				assertEquals((char) ('a' + random.nextInt(26)), segment.getChar(i));
-			}
-		}
-	}
-
-	@Test
-	public void doubleAccess() {
-		// test exceptions
-		{
-			try {
-				segment.putDouble(-1, 0.0);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.putDouble(PAGE_SIZE, 0.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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
-				segment.putDouble(i, random.nextDouble());
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
-				assertEquals(random.nextDouble(), segment.getDouble(i), 0.0);
-			}
-		}
-	}
-
-	// @Test
-	public void floatAccess() {
-		// test exceptions
-		{
-			try {
-				segment.putFloat(-1, 0.0f);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.putFloat(PAGE_SIZE, 0.0f);
-				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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
-				segment.putFloat(i, random.nextFloat());
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
-				assertEquals(random.nextFloat(), segment.getFloat(i), 0.0);
-			}
-		}
-	}
-
-	@Test
-	public void longAccess() {
-		// test exceptions
-		{
-			try {
-				segment.putLong(-1, 0L);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.putLong(PAGE_SIZE, 0L);
-				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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
-				segment.putLong(i, random.nextLong());
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 8; i += 8) {
-				assertEquals(random.nextLong(), segment.getLong(i));
-			}
-		}
-		
-		// test unaligned offsets
-		{
-			final long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
-				long value = random.nextLong();
-				segment.putLong(offset, value);
-			}
-			
-			random.setSeed(seed);
-			for (int offset = 0; offset < PAGE_SIZE - 8; offset += random.nextInt(24) + 8) {
-				long shouldValue = random.nextLong();
-				long isValue = segment.getLong(offset);
-				assertEquals(shouldValue, isValue);
-			}
-		}
-	}
-
-	@Test
-	public void intAccess() {
-		// test exceptions
-		{
-			try {
-				segment.putInt(-1, 0);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.putInt(PAGE_SIZE, 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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
-				segment.putInt(i, random.nextInt());
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 4; i += 4) {
-				assertEquals(random.nextInt(), segment.getInt(i));
-			}
-		}
-	}
-
-	@Test
-	public void shortAccess() {
-		// test exceptions
-		{
-			try {
-				segment.putShort(-1, (short) 0);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-
-			try {
-				segment.putShort(PAGE_SIZE, (short) 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(PAGE_SIZE);
-				fail("IndexOutOfBoundsException expected");
-			} catch (Exception e) {
-				assertTrue(e instanceof IndexOutOfBoundsException);
-			}
-		}
-
-		// test expected correct behavior
-		{
-			long seed = random.nextLong();
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
-				segment.putShort(i, (short) random.nextInt());
-			}
-
-			random.setSeed(seed);
-			for (int i = 0; i <= PAGE_SIZE - 2; i += 2) {
-				assertEquals((short) random.nextInt(), segment.getShort(i));
-			}
-		}
-	}
-	
-	@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());
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
index 30e417b..0a02f30 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/drivers/TestTaskContext.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.drivers;
 
 import org.apache.flink.api.common.ExecutionConfig;
@@ -25,10 +24,10 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.api.java.typeutils.runtime.RuntimeSerializerFactory;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.DriverStrategy;
 import org.apache.flink.runtime.operators.PactTaskContext;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
@@ -72,7 +71,7 @@ public class TestTaskContext<S, T> implements PactTaskContext<S, T> {
 	public TestTaskContext() {}
 	
 	public TestTaskContext(long memoryInBytes) {
-		this.memoryManager = new DefaultMemoryManager(memoryInBytes,1 ,32 * 1024, true);
+		this.memoryManager = new MemoryManager(memoryInBytes, 1, 32 * 1024, MemoryType.HEAP, true);
 		this.taskManageInfo = new TaskManagerRuntimeInfo("localhost", new Configuration());
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
index e3b697e..abcbf73 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
@@ -30,6 +30,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.util.MutableObjectIterator;
 
 import org.junit.Test;
@@ -239,7 +240,7 @@ public class CompactingHashTableTest {
 	private static List<MemorySegment> getMemory(int numSegments, int segmentSize) {
 		ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(numSegments);
 		for (int i = 0; i < numSegments; i++) {
-			list.add(new MemorySegment(new byte[segmentSize]));
+			list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize));
 		}
 		return list;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
index 52f6ffc..d0a6fc6 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableITCase.java
@@ -36,9 +36,8 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.UniformIntPairGenerator;
@@ -96,7 +95,7 @@ public class HashTableITCase {
 		this.pairProbeSideComparator = new IntPairComparator();
 		this.pairComparator = new IntPairPairComparator();
 		
-		this.memManager = new DefaultMemoryManager(32 * 1024 * 1024,1);
+		this.memManager = new MemoryManager(32 * 1024 * 1024,1);
 		this.ioManager = new IOManagerAsync();
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
index 0c656d6..9cb3606 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTablePerformanceComparison.java
@@ -25,6 +25,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
@@ -213,7 +214,7 @@ public class HashTablePerformanceComparison {
 		List<MemorySegment> memory = new ArrayList<MemorySegment>();
 		
 		for (int i = 0; i < numPages; i++) {
-			memory.add(new MemorySegment(new byte[pageSize]));
+			memory.add(MemorySegmentFactory.allocateUnpooledSegment(pageSize));
 		}
 		
 		return memory;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
index 0d8b81e..0bca22a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/HashTableTest.java
@@ -28,6 +28,7 @@ import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.TupleComparator;
 import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.util.MutableObjectIterator;
@@ -153,7 +154,7 @@ public class HashTableTest {
 	private static List<MemorySegment> getMemory(int numSegments, int segmentSize) {
 		ArrayList<MemorySegment> list = new ArrayList<MemorySegment>(numSegments);
 		for (int i = 0; i < numSegments; i++) {
-			list.add(new MemorySegment(new byte[segmentSize]));
+			list.add(MemorySegmentFactory.allocateUnpooledSegment(segmentSize));
 		}
 		return list;
 	}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
index 3dcf688..a71f028 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MemoryHashTableTest.java
@@ -27,6 +27,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.operators.testutils.UniformStringPairGenerator;
 import org.apache.flink.runtime.operators.testutils.types.IntList;
 import org.apache.flink.runtime.operators.testutils.types.IntListComparator;
@@ -737,7 +738,7 @@ public class MemoryHashTableTest {
 		List<MemorySegment> memory = new ArrayList<MemorySegment>();
 		
 		for (int i = 0; i < numPages; i++) {
-			memory.add(new MemorySegment(new byte[pageSize]));
+			memory.add(MemorySegmentFactory.allocateUnpooledSegment(pageSize));
 		}
 		
 		return memory;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
index c0f8f59..70c9427 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
@@ -28,9 +28,8 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.types.StringPair;
 import org.apache.flink.runtime.operators.testutils.types.StringPairComparator;
@@ -46,6 +45,7 @@ import org.junit.Test;
 import static org.junit.Assert.fail;
 
 public class MutableHashTablePerformanceBenchmark {
+	
 	private static final AbstractInvokable MEM_OWNER = new DummyInvokable();
 	
 	private MemoryManager memManager;
@@ -68,7 +68,7 @@ public class MutableHashTablePerformanceBenchmark {
 		this.pairProbeSideComparator = new StringPairComparator();
 		this.pairComparator = new StringPairPairComparator();
 		
-		this.memManager = new DefaultMemoryManager(64 * 1024 * 1024, 1);
+		this.memManager = new MemoryManager(64 * 1024 * 1024, 1);
 		this.ioManager = new IOManagerAsync();
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
index 0d5a26e..2da97e9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingHashMatchIteratorITCase.java
@@ -31,8 +31,7 @@ import org.apache.flink.api.java.record.functions.JoinFunction;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.TestData;
@@ -104,7 +103,7 @@ public class NonReusingHashMatchIteratorITCase {
 		this.pairRecordPairComparator = new IntPairRecordPairComparator();
 		this.recordPairPairComparator = new RecordIntPairPairComparator();
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
index 306a370..86f879a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/NonReusingReOpenableHashTableITCase.java
@@ -27,12 +27,12 @@ import org.apache.flink.api.common.typeutils.record.RecordPairComparator;
 import org.apache.flink.api.common.typeutils.record.RecordSerializer;
 import org.apache.flink.api.java.record.functions.JoinFunction;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.hash.HashTableITCase.ConstantsKeyValuePairsIterator;
 import org.apache.flink.runtime.operators.hash.MutableHashTable.HashBucketIterator;
 import org.apache.flink.runtime.operators.hash.NonReusingHashMatchIteratorITCase.RecordMatch;
@@ -120,7 +120,7 @@ public class NonReusingReOpenableHashTableITCase {
 		this.recordProbeSideComparator = new RecordComparator(keyPos, keyType);
 		this.pactRecordComparator = new HashTableITCase.RecordPairComparatorFirstInt();
 
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, true);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
index f770ca4..4fdff76 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingHashMatchIteratorITCase.java
@@ -38,8 +38,7 @@ import org.apache.flink.api.java.record.functions.JoinFunction;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.TestData;
@@ -104,7 +103,7 @@ public class ReusingHashMatchIteratorITCase {
 		this.pairRecordPairComparator = new IntPairRecordPairComparator();
 		this.recordPairPairComparator = new RecordIntPairPairComparator();
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
index d302487..ba5a325 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/ReusingReOpenableHashTableITCase.java
@@ -37,12 +37,12 @@ import org.apache.flink.api.common.typeutils.record.RecordPairComparator;
 import org.apache.flink.api.common.typeutils.record.RecordSerializer;
 import org.apache.flink.api.java.record.functions.JoinFunction;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.RecordMatch;
 import org.apache.flink.runtime.operators.hash.ReusingHashMatchIteratorITCase.RecordMatchRemovingJoin;
 import org.apache.flink.runtime.operators.hash.HashTableITCase.ConstantsKeyValuePairsIterator;
@@ -119,7 +119,7 @@ public class ReusingReOpenableHashTableITCase {
 		this.recordProbeSideComparator = new RecordComparator(keyPos, keyType);
 		this.pactRecordComparator = new HashTableITCase.RecordPairComparatorFirstInt();
 		
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, true);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1, PAGE_SIZE, MemoryType.HEAP, true);
 		this.ioManager = new IOManagerAsync();
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
index 41ade9f..bfb7cf2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIteratorTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.resettable;
 
 import java.util.ArrayList;
@@ -26,9 +25,7 @@ import org.junit.Assert;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.record.RecordSerializer;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
-import org.apache.flink.runtime.operators.resettable.BlockResettableMutableObjectIterator;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.MutableObjectIteratorWrapper;
 import org.apache.flink.types.IntValue;
@@ -38,8 +35,8 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
-public class BlockResettableMutableObjectIteratorTest
-{
+public class BlockResettableMutableObjectIteratorTest {
+	
 	private static final int MEMORY_CAPACITY = 3 * 128 * 1024;
 	
 	private static final int NUM_VALUES = 20000;
@@ -57,7 +54,7 @@ public class BlockResettableMutableObjectIteratorTest
 	@Before
 	public void startup() {
 		// set up IO and memory manager
-		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
+		this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
 		
 		// create test objects
 		this.objects = new ArrayList<Record>(20000);
@@ -83,8 +80,7 @@ public class BlockResettableMutableObjectIteratorTest
 	}
 
 	@Test
-	public void testSerialBlockResettableIterator() throws Exception
-	{
+	public void testSerialBlockResettableIterator() throws Exception {
 		try {
 			// create the resettable Iterator
 			final BlockResettableMutableObjectIterator<Record> iterator =
@@ -126,8 +122,7 @@ public class BlockResettableMutableObjectIteratorTest
 	}
 
 	@Test
-	public void testDoubleBufferedBlockResettableIterator() throws Exception
-	{
+	public void testDoubleBufferedBlockResettableIterator() throws Exception {
 		try {
 			// create the resettable Iterator
 			final BlockResettableMutableObjectIterator<Record> iterator =
@@ -170,8 +165,7 @@ public class BlockResettableMutableObjectIteratorTest
 	}
 
 	@Test
-	public void testTwelveFoldBufferedBlockResettableIterator() throws Exception
-	{
+	public void testTwelveFoldBufferedBlockResettableIterator() throws Exception {
 		try {
 			// create the resettable Iterator
 			final BlockResettableMutableObjectIterator<Record> iterator =
@@ -212,5 +206,4 @@ public class BlockResettableMutableObjectIteratorTest
 			Assert.fail("Test encountered an exception: " + ex.getMessage());
 		}
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
index 5641f29..d5de75e 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIteratorTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.resettable;
 
 import java.util.ArrayList;
@@ -27,18 +26,18 @@ import org.junit.Assert;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.record.RecordSerializer;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.types.Record;
+
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 
-public class NonReusingBlockResettableIteratorTest
-{
+public class NonReusingBlockResettableIteratorTest {
+	
 	private static final int MEMORY_CAPACITY = 3 * 128 * 1024;
 	
 	private static final int NUM_VALUES = 20000;
@@ -54,7 +53,7 @@ public class NonReusingBlockResettableIteratorTest
 	@Before
 	public void startup() {
 		// set up IO and memory manager
-		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
+		this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
 		
 		// create test objects
 		this.objects = new ArrayList<Record>(20000);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
index f0fadfe..7dbd2fb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIteratorTest.java
@@ -16,17 +16,16 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.resettable;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.record.RecordSerializer;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.types.IntValue;
 import org.apache.flink.types.Record;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -36,9 +35,8 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
-
-public class ReusingBlockResettableIteratorTest
-{
+public class ReusingBlockResettableIteratorTest {
+	
 	private static final int MEMORY_CAPACITY = 3 * 128 * 1024;
 	
 	private static final int NUM_VALUES = 20000;
@@ -54,7 +52,7 @@ public class ReusingBlockResettableIteratorTest
 	@Before
 	public void startup() {
 		// set up IO and memory manager
-		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 1);
+		this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
 		
 		// create test objects
 		this.objects = new ArrayList<Record>(20000);
@@ -197,5 +195,4 @@ public class ReusingBlockResettableIteratorTest
 		// close the iterator
 		iterator.close();
 	}
-
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
index 4db520e..0ab9a54 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIteratorTest.java
@@ -24,13 +24,14 @@ import java.util.NoSuchElementException;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.base.IntValueSerializer;
+import org.apache.flink.core.memory.MemoryType;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.types.IntValue;
+
 import org.junit.Assert;
 import org.junit.After;
 import org.junit.Before;
@@ -56,7 +57,7 @@ public class SpillingResettableIteratorTest {
 	@Before
 	public void startup() {
 		// set up IO and memory manager
-		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 32 * 1024);
+		this.memman = new MemoryManager(MEMORY_CAPACITY, 1, 32 * 1024, MemoryType.HEAP, true);
 		this.ioman = new IOManagerAsync();
 
 		// create test objects

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
index fa72bbf..c64db54 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIteratorTest.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.runtime.operators.resettable;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -24,8 +23,7 @@ import org.apache.flink.api.common.typeutils.record.RecordSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.MutableObjectIteratorWrapper;
 import org.apache.flink.types.IntValue;
@@ -56,7 +54,7 @@ public class SpillingResettableMutableObjectIteratorTest {
 	@Before
 	public void startup() {
 		// set up IO and memory manager
-		this.memman = new DefaultMemoryManager(MEMORY_CAPACITY, 32 * 1024);
+		this.memman = new MemoryManager(MEMORY_CAPACITY, 1);
 		this.ioman = new IOManagerAsync();
 
 		// create test objects

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
index d4da9d3..0c0e836 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/AbstractSortMergeOuterJoinIteratorITCase.java
@@ -36,8 +36,7 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.testutils.CollectionIterator;
 import org.apache.flink.runtime.operators.testutils.DiscardingOutputCollector;
@@ -53,6 +52,7 @@ import org.apache.flink.runtime.operators.testutils.TestData.TupleGeneratorItera
 import org.apache.flink.runtime.util.ResettableMutableObjectIterator;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
@@ -103,7 +103,7 @@ public abstract class AbstractSortMergeOuterJoinIteratorITCase {
 		comparator2 = typeInfo2.createComparator(new int[]{0}, new boolean[]{true}, 0, config);
 		pairComp = new GenericPairComparator<>(comparator1, comparator2);
 
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 	}
 
@@ -299,7 +299,7 @@ public abstract class AbstractSortMergeOuterJoinIteratorITCase {
 
 		TypePairComparator<Tuple2<Integer, String>, Tuple2<Integer, String>> pairComparator = new GenericPairComparator<>(comparator1, comparator2);
 
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 
 		final int DUPLICATE_KEY = 13;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
index 0ba9823..75593b8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMergerITCase.java
@@ -37,8 +37,7 @@ import org.apache.flink.configuration.Configuration;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.TestData;
 import org.apache.flink.runtime.operators.testutils.TestData.Key;
@@ -80,7 +79,7 @@ public class CombiningUnilateralSortMergerITCase {
 	@SuppressWarnings("unchecked")
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 		
 		this.serializerFactory = RecordSerializerFactory.get();
@@ -190,7 +189,7 @@ public class CombiningUnilateralSortMergerITCase {
 	{
 		final Hashtable<TestData.Key, Integer> countTable = new Hashtable<TestData.Key, Integer>(KEY_MAX);
 		for (int i = 1; i <= KEY_MAX; i++) {
-			countTable.put(new TestData.Key(i), Integer.valueOf(0));
+			countTable.put(new TestData.Key(i), 0);
 		}
 
 		// comparator

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
index 46a1632..5aa9efb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortITCase.java
@@ -27,8 +27,7 @@ import org.apache.flink.api.common.typeutils.record.RecordSerializerFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.runtime.operators.testutils.RandomIntPairGenerator;
 import org.apache.flink.runtime.operators.testutils.TestData;
@@ -82,7 +81,7 @@ public class ExternalSortITCase {
 	@SuppressWarnings("unchecked")
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 		
 		this.pactRecordSerializer = RecordSerializerFactory.get();

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
index 86be614..951ce30 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSortLargeRecordsITCase.java
@@ -32,8 +32,7 @@ import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.DefaultMemoryManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.testutils.DummyInvokable;
 import org.apache.flink.util.MutableObjectIterator;
 import org.junit.After;
@@ -66,7 +65,7 @@ public class ExternalSortLargeRecordsITCase {
 
 	@Before
 	public void beforeTest() {
-		this.memoryManager = new DefaultMemoryManager(MEMORY_SIZE, 1);
+		this.memoryManager = new MemoryManager(MEMORY_SIZE, 1);
 		this.ioManager = new IOManagerAsync();
 	}
 


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

Posted by se...@apache.org.
[FLINK-1320] [core] Add an off-heap variant of the managed memory

This closes #1093


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

Branch: refs/heads/master
Commit: 655a891d929db9d858bb5c2edf54419f2b0d3ace
Parents: 1800434
Author: Stephan Ewen <se...@apache.org>
Authored: Sun Aug 30 22:36:46 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200

----------------------------------------------------------------------
 .../flink/configuration/ConfigConstants.java    |   14 +-
 .../flink/core/memory/HeapMemorySegment.java    |  203 ++
 .../flink/core/memory/HybridMemorySegment.java  |  466 ++++
 .../apache/flink/core/memory/MemorySegment.java |  938 ++++---
 .../flink/core/memory/MemorySegmentFactory.java |  211 ++
 .../apache/flink/core/memory/MemoryType.java    |   35 +
 .../apache/flink/core/memory/MemoryUtils.java   |   34 +-
 .../common/typeutils/ComparatorTestBase.java    |   10 +-
 .../flink/core/memory/CrossSegmentTypeTest.java |  356 +++
 .../core/memory/EndiannessAccessChecks.java     |  183 ++
 .../core/memory/HeapMemorySegmentTest.java      |   71 +
 .../memory/HybridOffHeapMemorySegmentTest.java  |   84 +
 .../memory/HybridOnHeapMemorySegmentTest.java   |   82 +
 .../core/memory/MemorySegmentChecksTest.java    |  135 +
 .../core/memory/MemorySegmentTestBase.java      | 2571 ++++++++++++++++++
 .../memory/MemorySegmentUndersizedTest.java     | 1367 ++++++++++
 .../memory/OperationsOnFreedSegmentTest.java    | 1195 ++++++++
 .../benchmarks/CoreMemorySegmentOutView.java    |  360 +++
 .../LongSerializationSpeedBenchmark.java        |  232 ++
 .../benchmarks/MemorySegmentSpeedBenchmark.java | 1633 +++++++++++
 .../benchmarks/PureHeapMemorySegment.java       |  466 ++++
 .../PureHeapMemorySegmentOutView.java           |  359 +++
 .../benchmarks/PureHybridMemorySegment.java     |  887 ++++++
 .../PureHybridMemorySegmentOutView.java         |  359 +++
 .../benchmarks/PureOffHeapMemorySegment.java    |  790 ++++++
 .../PureOffHeapMemorySegmentOutView.java        |  359 +++
 .../StringSerializationSpeedBenchmark.java      |  207 ++
 .../apache/flink/types/NormalizableKeyTest.java |   26 +-
 .../flink/runtime/execution/Environment.java    |    2 +-
 .../runtime/io/disk/FileChannelInputView.java   |    4 +-
 .../runtime/io/disk/FileChannelOutputView.java  |    4 +-
 .../runtime/io/disk/RandomAccessInputView.java  |    2 +-
 .../runtime/io/disk/RandomAccessOutputView.java |    2 +-
 .../io/disk/SeekableFileChannelInputView.java   |    4 +-
 .../io/disk/SimpleCollectingOutputView.java     |    2 +-
 .../flink/runtime/io/disk/SpillingBuffer.java   |    2 +-
 .../disk/iomanager/ChannelReaderInputView.java  |    2 +-
 .../disk/iomanager/ChannelWriterOutputView.java |    2 +-
 .../runtime/io/network/NetworkEnvironment.java  |    3 +-
 .../AdaptiveSpanningRecordDeserializer.java     |    6 +-
 .../api/serialization/EventSerializer.java      |    6 +-
 ...llingAdaptiveSpanningRecordDeserializer.java |    6 +-
 .../flink/runtime/io/network/buffer/Buffer.java |    5 +-
 .../io/network/buffer/NetworkBufferPool.java    |   30 +-
 .../netty/PartitionRequestClientHandler.java    |    4 +-
 .../SpilledSubpartitionViewSyncIO.java          |    3 +-
 .../iterative/io/SerializedUpdateBuffer.java    |    4 +-
 .../runtime/memory/AbstractPagedInputView.java  |  568 ++++
 .../runtime/memory/AbstractPagedOutputView.java |  414 +++
 .../runtime/memory/ListMemorySegmentSource.java |   48 +
 .../memory/MemoryAllocationException.java       |   44 +
 .../flink/runtime/memory/MemoryManager.java     |  700 +++++
 .../memorymanager/AbstractPagedInputView.java   |  566 ----
 .../memorymanager/AbstractPagedOutputView.java  |  416 ---
 .../memorymanager/CheckedMemorySegment.java     |  407 ---
 .../memorymanager/DefaultMemoryManager.java     |  490 ----
 .../memorymanager/ListMemorySegmentSource.java  |   48 -
 .../MemoryAllocationException.java              |   44 -
 .../runtime/memorymanager/MemoryManager.java    |  134 -
 .../memorymanager/SimpleMemorySegment.java      |  329 ---
 .../memorymanager/UnsafeMemorySegment.java      |  391 ---
 .../operators/AbstractOuterJoinDriver.java      |    2 +-
 .../flink/runtime/operators/CrossDriver.java    |    2 +-
 .../runtime/operators/FullOuterJoinDriver.java  |    2 +-
 .../operators/GroupReduceCombineDriver.java     |    2 +-
 .../flink/runtime/operators/JoinDriver.java     |    2 +-
 .../runtime/operators/LeftOuterJoinDriver.java  |    2 +-
 .../runtime/operators/PactTaskContext.java      |    2 +-
 .../runtime/operators/ReduceCombineDriver.java  |    2 +-
 .../runtime/operators/RegularPactTask.java      |    2 +-
 .../runtime/operators/RightOuterJoinDriver.java |    2 +-
 .../flink/runtime/operators/TempBarrier.java    |    6 +-
 .../chaining/GroupCombineChainedDriver.java     |    2 +-
 .../SynchronousChainedCombineDriver.java        |    2 +-
 .../operators/hash/CompactingHashTable.java     |    2 +-
 .../operators/hash/HashMatchIteratorBase.java   |    4 +-
 .../runtime/operators/hash/HashPartition.java   |    4 +-
 .../operators/hash/InMemoryPartition.java       |    6 +-
 .../NonReusingBuildFirstHashMatchIterator.java  |    4 +-
 ...ngBuildFirstReOpenableHashMatchIterator.java |    4 +-
 .../NonReusingBuildSecondHashMatchIterator.java |    4 +-
 ...gBuildSecondReOpenableHashMatchIterator.java |    4 +-
 .../ReusingBuildFirstHashMatchIterator.java     |    4 +-
 ...ngBuildFirstReOpenableHashMatchIterator.java |    4 +-
 .../ReusingBuildSecondHashMatchIterator.java    |    4 +-
 ...gBuildSecondReOpenableHashMatchIterator.java |    4 +-
 .../AbstractBlockResettableIterator.java        |    6 +-
 .../BlockResettableMutableObjectIterator.java   |    4 +-
 .../NonReusingBlockResettableIterator.java      |    4 +-
 .../ReusingBlockResettableIterator.java         |    4 +-
 .../resettable/SpillingResettableIterator.java  |    6 +-
 ...SpillingResettableMutableObjectIterator.java |    6 +-
 .../sort/AbstractMergeInnerJoinIterator.java    |    4 +-
 .../operators/sort/AbstractMergeIterator.java   |    4 +-
 .../sort/AbstractMergeOuterJoinIterator.java    |    4 +-
 .../sort/CombiningUnilateralSortMerger.java     |    4 +-
 .../operators/sort/FixedLengthRecordSorter.java |   12 +-
 .../operators/sort/LargeRecordHandler.java      |    2 +-
 .../sort/NonReusingMergeInnerJoinIterator.java  |    4 +-
 .../sort/NonReusingMergeOuterJoinIterator.java  |    4 +-
 .../operators/sort/NormalizedKeySorter.java     |   18 +-
 .../sort/ReusingMergeInnerJoinIterator.java     |    4 +-
 .../sort/ReusingMergeOuterJoinIterator.java     |    4 +-
 .../operators/sort/UnilateralSortMerger.java    |    4 +-
 .../operators/util/CoGroupTaskIterator.java     |    2 +-
 .../operators/util/JoinTaskIterator.java        |    2 +-
 .../runtime/taskmanager/RuntimeEnvironment.java |    2 +-
 .../apache/flink/runtime/taskmanager/Task.java  |    2 +-
 .../runtime/util/EnvironmentInformation.java    |   19 +-
 .../NetworkEnvironmentConfiguration.scala       |    4 +-
 .../flink/runtime/taskmanager/TaskManager.scala |   99 +-
 .../flink/runtime/io/disk/ChannelViewsTest.java |    6 +-
 .../io/disk/FileChannelStreamsITCase.java       |    7 +-
 .../runtime/io/disk/FileChannelStreamsTest.java |    8 +-
 .../disk/SeekableFileChannelInputViewTest.java  |   24 +-
 .../runtime/io/disk/SpillingBufferTest.java     |   31 +-
 .../AsynchronousFileIOChannelTest.java          |    8 +-
 .../BufferFileWriterFileSegmentReaderTest.java  |    7 +-
 .../iomanager/BufferFileWriterReaderTest.java   |    4 +-
 .../io/disk/iomanager/IOManagerAsyncTest.java   |    5 +-
 .../io/disk/iomanager/IOManagerITCase.java      |    8 +-
 .../IOManagerPerformanceBenchmark.java          |    6 +-
 .../io/network/NetworkEnvironmentTest.java      |    4 +-
 .../api/serialization/PagedViewsTest.java       |   15 +-
 .../SpanningRecordSerializationTest.java        |    4 +-
 .../SpanningRecordSerializerTest.java           |    7 +-
 .../io/network/api/writer/RecordWriterTest.java |    9 +-
 .../network/buffer/BufferPoolFactoryTest.java   |    3 +-
 .../runtime/io/network/buffer/BufferTest.java   |    5 +-
 .../io/network/buffer/LocalBufferPoolTest.java  |    3 +-
 .../network/buffer/NetworkBufferPoolTest.java   |    5 +-
 .../netty/NettyMessageSerializationTest.java    |    4 +-
 .../IteratorWrappingTestSingleInputGate.java    |    4 +-
 .../consumer/LocalInputChannelTest.java         |    3 +-
 .../partition/consumer/SingleInputGateTest.java |    5 +-
 .../network/serialization/LargeRecordsTest.java |    6 +-
 .../io/network/util/TestBufferFactory.java      |    6 +-
 .../memory/MemoryManagerLazyAllocationTest.java |    7 +-
 .../flink/runtime/memory/MemoryManagerTest.java |    7 +-
 .../runtime/memory/MemorySegmentSimpleTest.java |  576 ++++
 .../memory/MemorySegmentSpeedBenchmark.java     |  867 ------
 .../flink/runtime/memory/MemorySegmentTest.java |  575 ----
 .../operators/drivers/TestTaskContext.java      |    7 +-
 .../operators/hash/CompactingHashTableTest.java |    3 +-
 .../runtime/operators/hash/HashTableITCase.java |    7 +-
 .../hash/HashTablePerformanceComparison.java    |    3 +-
 .../runtime/operators/hash/HashTableTest.java   |    3 +-
 .../operators/hash/MemoryHashTableTest.java     |    3 +-
 .../MutableHashTablePerformanceBenchmark.java   |    8 +-
 .../hash/NonReusingHashMatchIteratorITCase.java |    5 +-
 .../NonReusingReOpenableHashTableITCase.java    |    8 +-
 .../hash/ReusingHashMatchIteratorITCase.java    |    5 +-
 .../hash/ReusingReOpenableHashTableITCase.java  |    8 +-
 ...lockResettableMutableObjectIteratorTest.java |   21 +-
 .../NonReusingBlockResettableIteratorTest.java  |   11 +-
 .../ReusingBlockResettableIteratorTest.java     |   13 +-
 .../SpillingResettableIteratorTest.java         |    7 +-
 ...lingResettableMutableObjectIteratorTest.java |    6 +-
 ...bstractSortMergeOuterJoinIteratorITCase.java |    8 +-
 .../CombiningUnilateralSortMergerITCase.java    |    7 +-
 .../operators/sort/ExternalSortITCase.java      |    5 +-
 .../sort/ExternalSortLargeRecordsITCase.java    |    5 +-
 .../sort/FixedLengthRecordSorterTest.java       |   14 +-
 .../sort/LargeRecordHandlerITCase.java          |   28 +-
 .../operators/sort/LargeRecordHandlerTest.java  |   21 +-
 ...ReusingSortMergeInnerJoinIteratorITCase.java |   17 +-
 ...ReusingSortMergeOuterJoinIteratorITCase.java |    2 +-
 .../operators/sort/NormalizedKeySorterTest.java |   15 +-
 ...ReusingSortMergeInnerJoinIteratorITCase.java |   17 +-
 ...ReusingSortMergeOuterJoinIteratorITCase.java |    2 +-
 .../testutils/BinaryOperatorTestBase.java       |    5 +-
 .../operators/testutils/DriverTestBase.java     |    5 +-
 .../operators/testutils/MockEnvironment.java    |    9 +-
 .../operators/testutils/TaskTestBase.java       |    2 +-
 .../testutils/UnaryOperatorTestBase.java        |    5 +-
 .../runtime/operators/util/BloomFilterTest.java |    5 +-
 .../operators/util/HashVsSortMiniBenchmark.java |    6 +-
 .../runtime/taskmanager/TaskAsyncCallTest.java  |    2 +-
 ...askManagerComponentsStartupShutdownTest.java |    8 +-
 .../flink/runtime/taskmanager/TaskTest.java     |    2 +-
 .../util/DataInputOutputSerializerTest.java     |    4 +-
 .../testingUtils/TestingTaskManager.scala       |    4 +-
 .../streaming/runtime/io/BufferSpiller.java     |    3 +-
 .../consumer/StreamTestSingleInputGate.java     |   18 +-
 .../io/BarrierBufferMassiveRandomTest.java      |    5 +-
 .../streaming/runtime/io/BarrierBufferTest.java |    8 +-
 .../runtime/io/BarrierTrackerTest.java          |    4 +-
 .../streaming/runtime/io/BufferSpillerTest.java |    3 +-
 .../runtime/io/StreamRecordWriterTest.java      |    7 +-
 .../runtime/tasks/StreamMockEnvironment.java    |   11 +-
 .../runtime/tasks/StreamTaskTestHarness.java    |    2 +-
 .../tez/runtime/TezRuntimeEnvironment.java      |   11 +-
 .../org/apache/flink/tez/runtime/TezTask.java   |    2 +-
 .../HashTableRecordWidthCombinations.java       |    3 +-
 .../flink/test/manual/MassiveStringSorting.java |    8 +-
 .../test/manual/MassiveStringValueSorting.java  |   19 +-
 .../misc/MassiveCaseClassSortingITCase.scala    |    4 +-
 .../scala/runtime/CaseClassComparatorTest.scala |   30 +-
 .../org/apache/flink/yarn/YarnTaskManager.scala |    4 +-
 pom.xml                                         |    2 +-
 tools/maven/checkstyle.xml                      |    2 +-
 201 files changed, 16226 insertions(+), 5096 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 9ec71d2..bbaf71a 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -125,9 +125,14 @@ public final class ConfigConstants {
 	public static final String TASK_MANAGER_MEMORY_FRACTION_KEY = "taskmanager.memory.fraction";
 
 	/**
-	 * The key for the config parameter defining whether the memory manager allocates memory lazy.
+	 * The fraction of off-heap memory relative to the heap size.
 	 */
-	public static final String TASK_MANAGER_MEMORY_LAZY_ALLOCATION_KEY = "taskmanager.memory.lazyalloc";
+	public static final String TASK_MANAGER_MEMORY_OFF_HEAP_RATIO_KEY = "taskmanager.memory.off-heap-ratio";
+	
+	/**
+	 * The config parameter defining the memory allocation method (JVM heap or off-heap).
+	*/
+	public static final String TASK_MANAGER_MEMORY_OFF_HEAP_KEY = "taskmanager.memory.off-heap";
 
 	/**
 	 * The config parameter defining the number of buffers used in the network stack. This defines the
@@ -543,6 +548,11 @@ public final class ConfigConstants {
 	public static final float DEFAULT_MEMORY_MANAGER_MEMORY_FRACTION = 0.7f;
 
 	/**
+	 * The default ratio of heap to off-heap memory, when the TaskManager is started with off-heap memory.
+	 */
+	public static final float DEFAULT_MEMORY_MANAGER_MEMORY_OFF_HEAP_RATIO = 3.0f;
+	
+	/**
 	 * Default number of buffers used in the network stack.
 	 */
 	public static final int DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS = 2048;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java
new file mode 100644
index 0000000..0685d59
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/HeapMemorySegment.java
@@ -0,0 +1,203 @@
+/*
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Objects;
+
+/**
+ * This class represents a piece of heap memory managed by Flink.
+ * The segment is backed by a byte array and features random put and get methods for the basic types,
+ * as well as compare and swap methods.
+ * <p>
+ * This class specialized byte access and byte copy calls for heap memory, while reusing the
+ * multi-byte type accesses and cross-segment operations from the MemorySegment.
+ * <p>
+ * Note that memory segments should usually not be allocated manually, but rather through the
+ * {@link MemorySegmentFactory}.
+ */
+public final class HeapMemorySegment extends MemorySegment {
+
+	/** An extra reference to the heap memory, so we can let byte array checks fail 
+	 *  by the built-in checks automatically without extra checks */
+	private byte[] memory;
+
+	/**
+	 * Creates a new memory segment that represents the data in the given byte array.
+	 * The owner of this memory segment is null.
+	 *
+	 * @param memory The byte array that holds the data.
+	 */
+	HeapMemorySegment(byte[] memory) {
+		this(memory, null);
+	}
+	
+	/**
+	 * Creates a new memory segment that represents the data in the given byte array.
+	 * The memory segment references the given owner.
+	 *
+	 * @param memory The byte array that holds the data.
+	 * @param owner The owner referenced by the memory segment.
+	 */
+	HeapMemorySegment(byte[] memory, Object owner) {
+		super(Objects.requireNonNull(memory), owner);
+		this.memory = memory;
+	}
+	
+	// -------------------------------------------------------------------------
+	//  MemorySegment operations
+	// -------------------------------------------------------------------------
+
+	@Override
+	public void free() {
+		super.free();
+		this.memory = null;
+	}
+
+	@Override
+	public ByteBuffer wrap(int offset, int length) {
+		try {
+			return ByteBuffer.wrap(this.memory, offset, length);
+		}
+		catch (NullPointerException e) {
+			throw new IllegalStateException("segment has been freed");
+		}
+	}
+
+	/**
+	 * Gets the byte array that backs this memory segment.
+	 *
+	 * @return The byte array that backs this memory segment, or null, if the segment has been freed.
+	 */
+	public byte[] getArray() {
+		return this.heapMemory;
+	}
+	
+	// ------------------------------------------------------------------------
+	//                    Random Access get() and put() methods
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public final byte get(int index) {
+		return this.memory[index];
+	}
+
+	@Override
+	public final void put(int index, byte b) {
+		this.memory[index] = b;
+	}
+
+	@Override
+	public final void get(int index, byte[] dst) {
+		get(index, dst, 0, dst.length);
+	}
+
+	@Override
+	public final void put(int index, byte[] src) {
+		put(index, src, 0, src.length);
+	}
+
+	@Override
+	public final void get(int index, byte[] dst, int offset, int length) {
+		// system arraycopy does the boundary checks anyways, no need to check extra
+		System.arraycopy(this.memory, index, dst, offset, length);
+	}
+
+	@Override
+	public final void put(int index, byte[] src, int offset, int length) {
+		// system arraycopy does the boundary checks anyways, no need to check extra
+		System.arraycopy(src, offset, this.memory, index, length);
+	}
+
+	@Override
+	public final boolean getBoolean(int index) {
+		return this.memory[index] != 0;
+	}
+
+	@Override
+	public final void putBoolean(int index, boolean value) {
+		this.memory[index] = (byte) (value ? 1 : 0);
+	}
+
+	// -------------------------------------------------------------------------
+	//                     Bulk Read and Write Methods
+	// -------------------------------------------------------------------------
+
+	@Override
+	public final void get(DataOutput out, int offset, int length) throws IOException {
+		out.write(this.memory, offset, length);
+	}
+
+	@Override
+	public final void put(DataInput in, int offset, int length) throws IOException {
+		in.readFully(this.memory, offset, length);
+	}
+
+	@Override
+	public final void get(int offset, ByteBuffer target, int numBytes) {
+		// ByteBuffer performs the boundary checks
+		target.put(this.memory, offset, numBytes);
+	}
+
+	@Override
+	public final void put(int offset, ByteBuffer source, int numBytes) {
+		// ByteBuffer performs the boundary checks
+		source.get(this.memory, offset, numBytes);
+	}
+
+	// -------------------------------------------------------------------------
+	//                             Factoring
+	// -------------------------------------------------------------------------
+
+	/**
+	 * A memory segment factory that produces heap memory segments. Note that this factory does not
+	 * support to allocate off-heap memory.
+	 */
+	public static final class HeapMemorySegmentFactory implements MemorySegmentFactory.Factory {
+
+		@Override
+		public HeapMemorySegment wrap(byte[] memory) {
+			return new HeapMemorySegment(memory);
+		}
+
+		@Override
+		public HeapMemorySegment allocateUnpooledSegment(int size, Object owner) {
+			return new HeapMemorySegment(new byte[size], owner);
+		}
+
+		@Override
+		public HeapMemorySegment wrapPooledHeapMemory(byte[] memory, Object owner) {
+			return new HeapMemorySegment(memory, owner);
+		}
+
+		@Override
+		public HeapMemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner) {
+			throw new UnsupportedOperationException(
+					"The MemorySegment factory was not initialized for off-heap memory.");
+		}
+
+		/** prevent external instantiation */
+		HeapMemorySegmentFactory() {}
+	};
+
+	public static final HeapMemorySegmentFactory FACTORY = new HeapMemorySegmentFactory();
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java
new file mode 100644
index 0000000..f68723b
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/HybridMemorySegment.java
@@ -0,0 +1,466 @@
+/*
+ * 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 java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+
+/**
+ * This class represents a piece of memory managed by Flink. The memory can be on-heap or off-heap,
+ * this is transparently handled by this class.
+ * <p>
+ * This class specialized byte access and byte copy calls for heap memory, while reusing the
+ * multi-byte type accesses and cross-segment operations from the MemorySegment.
+ * <p>
+ * This class subsumes the functionality of the {@link org.apache.flink.core.memory.HeapMemorySegment}, 
+ * but is a bit less efficient for operations on individual bytes.
+ * <p>
+ * Note that memory segments should usually not be allocated manually, but rather through the
+ * {@link MemorySegmentFactory}.
+ */
+public final class HybridMemorySegment extends MemorySegment {
+	
+	/** The direct byte buffer that allocated the off-heap memory. This memory segment holds a reference
+	 * to that buffer, so as long as this memory segment lives, the memory will not be released. */
+	private final ByteBuffer offHeapBuffer;
+
+	/**
+	 * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+	 * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+	 * otherwise this method with throw an IllegalArgumentException.
+	 * <p>
+	 * The owner referenced by this memory segment is null.
+	 *
+	 * @param buffer The byte buffer whose memory is represented by this memory segment.
+	 * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+	 */
+	HybridMemorySegment(ByteBuffer buffer) {
+		this(buffer, null);
+	}
+	
+	/**
+	 * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+	 * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+	 * otherwise this method with throw an IllegalArgumentException.
+	 * <p>
+	 * The memory segment references the given owner.
+	 *
+	 * @param buffer The byte buffer whose memory is represented by this memory segment.
+	 * @param owner The owner references by this memory segment.
+	 * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+	 */
+	HybridMemorySegment(ByteBuffer buffer, Object owner) {
+		super(checkBufferAndGetAddress(buffer), buffer.capacity(), owner);
+		this.offHeapBuffer = buffer;
+	}
+
+	/**
+	 * Creates a new memory segment that represents the memory of the byte array.
+	 * <p>
+	 * The owner referenced by this memory segment is null.
+	 *
+	 * @param buffer The byte array whose memory is represented by this memory segment.
+	 */
+	HybridMemorySegment(byte[] buffer) {
+		this(buffer, null);
+	}
+
+	/**
+	 * Creates a new memory segment that represents the memory of the byte array.
+	 * <p>
+	 * The memory segment references the given owner.
+	 *
+	 * @param buffer The byte array whose memory is represented by this memory segment.
+	 * @param owner The owner references by this memory segment.
+	 */
+	HybridMemorySegment(byte[] buffer, Object owner) {
+		super(buffer, owner);
+		this.offHeapBuffer = null;
+	}
+
+	// -------------------------------------------------------------------------
+	//  MemorySegment operations
+	// -------------------------------------------------------------------------
+	
+	public byte[] getArray() {
+		if (heapMemory != null) {
+			return heapMemory;
+		} else {
+			throw new IllegalStateException("Memory segment does not represent heap memory");
+		}
+	}
+
+	/**
+	 * Gets the buffer that owns the memory of this memory segment.
+	 *
+	 * @return The byte buffer that owns the memory of this memory segment.
+	 */
+	public ByteBuffer getOffHeapBuffer() {
+		if (offHeapBuffer != null) {
+			return offHeapBuffer;
+		} else {
+			throw new IllegalStateException("Memory segment does not represent off heap memory");
+		}
+	}
+
+	@Override
+	public ByteBuffer wrap(int offset, int length) {
+		if (address <= addressLimit) {
+			if (heapMemory != null) {
+				return ByteBuffer.wrap(heapMemory, offset, length);
+			}
+			else {
+				try {
+					ByteBuffer wrapper = offHeapBuffer.duplicate();
+					wrapper.limit(offset + length);
+					wrapper.position(offset);
+					return wrapper;
+				}
+				catch (IllegalArgumentException e) {
+					throw new IndexOutOfBoundsException();
+				}
+			}
+		}
+		else {
+			throw new IllegalStateException("segment has been freed");
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Random Access get() and put() methods
+	// ------------------------------------------------------------------------
+	
+	@Override
+	public byte get(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos < addressLimit) {
+			return UNSAFE.getByte(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	@Override
+	public void put(int index, byte b) {
+		final long pos = address + index;
+		if (index >= 0 && pos < addressLimit) {
+			UNSAFE.putByte(heapMemory, pos, b);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	@Override
+	public void get(int index, byte[] dst) {
+		get(index, dst, 0, dst.length);
+	}
+
+	@Override
+	public void put(int index, byte[] src) {
+		put(index, src, 0, src.length);
+	}
+
+	@Override
+	public void get(int index, byte[] dst, int offset, int length) {
+		// check the byte array offset and length and the status
+		if ( (offset | length | (offset + length) | (dst.length - (offset + length))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - length) {
+			final long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+			UNSAFE.copyMemory(heapMemory, pos, dst, arrayAddress, length);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	@Override
+	public void put(int index, byte[] src, int offset, int length) {
+		// check the byte array offset and length
+		if ((offset | length | (offset + length) | (src.length - (offset + length))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+		
+		final long pos = address + index;
+
+		if (index >= 0 && pos <= addressLimit - length) {
+			final long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+			UNSAFE.copyMemory(src, arrayAddress, heapMemory, pos, length);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	@Override
+	public boolean getBoolean(int index) {
+		return get(index) != 0;
+	}
+
+	@Override
+	public void putBoolean(int index, boolean value) {
+		put(index, (byte) (value ? 1 : 0));
+	}
+
+	// -------------------------------------------------------------------------
+	//  Bulk Read and Write Methods
+	// -------------------------------------------------------------------------
+
+	@Override
+	public final void get(DataOutput out, int offset, int length) throws IOException {
+		if (address <= addressLimit) {
+			if (heapMemory != null) {
+				out.write(heapMemory, offset, length);
+			}
+			else {
+				while (length >= 8) {
+					out.writeLong(getLongBigEndian(offset));
+					offset += 8;
+					length -= 8;
+				}
+		
+				while (length > 0) {
+					out.writeByte(get(offset));
+					offset++;
+					length--;
+				}
+			}
+		}
+		else {
+			throw new IllegalStateException("segment has been freed");
+		}
+	}
+
+	@Override
+	public final void put(DataInput in, int offset, int length) throws IOException {
+		if (address <= addressLimit) {
+			if (heapMemory != null) {
+				in.readFully(heapMemory, offset, length);
+			}
+			else {
+				while (length >= 8) {
+					putLongBigEndian(offset, in.readLong());
+					offset += 8;
+					length -= 8;
+				}
+				while (length > 0) {
+					put(offset, in.readByte());
+					offset++;
+					length--;
+				}
+			}
+		}
+		else {
+			throw new IllegalStateException("segment has been freed");
+		}
+	}
+
+	@Override
+	public final void get(int offset, ByteBuffer target, int numBytes) {
+		// check the byte array offset and length
+		if ((offset | numBytes | (offset + numBytes)) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		final int targetOffset = target.position();
+		final int remaining = target.remaining();
+
+		if (remaining < numBytes) {
+			throw new BufferOverflowException();
+		}
+
+		if (target.isDirect()) {
+			// copy to the target memory directly
+			final long targetPointer = getAddress(target) + targetOffset;
+			final long sourcePointer = address + offset;
+
+			if (sourcePointer <= addressLimit - numBytes) {
+				UNSAFE.copyMemory(heapMemory, sourcePointer, null, targetPointer, numBytes);
+				target.position(targetOffset + numBytes);
+			}
+			else if (address > addressLimit) {
+				throw new IllegalStateException("segment has been freed");
+			}
+			else {
+				throw new IndexOutOfBoundsException();
+			}
+		}
+		else if (target.hasArray()) {
+			// move directly into the byte array
+			get(offset, target.array(), targetOffset + target.arrayOffset(), numBytes);
+
+			// this must be after the get() call to ensue that the byte buffer is not
+			// modified in case the call fails
+			target.position(targetOffset + numBytes);
+		}
+		else {
+			// neither heap buffer nor direct buffer
+			while (target.hasRemaining()) {
+				target.put(get(offset++));
+			}
+		}
+	}
+
+	@Override
+	public final void put(int offset, ByteBuffer source, int numBytes) {
+		// check the byte array offset and length
+		if ((offset | numBytes | (offset + numBytes)) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		final int sourceOffset = source.position();
+		final int remaining = source.remaining();
+
+		if (remaining < numBytes) {
+			throw new BufferUnderflowException();
+		}
+
+		if (source.isDirect()) {
+			// copy to the target memory directly
+			final long sourcePointer = getAddress(source) + sourceOffset;
+			final long targetPointer = address + offset;
+
+			if (targetPointer <= addressLimit - numBytes) {
+				UNSAFE.copyMemory(null, sourcePointer, heapMemory, targetPointer, numBytes);
+				source.position(sourceOffset + numBytes);
+			}
+			else if (address > addressLimit) {
+				throw new IllegalStateException("segment has been freed");
+			}
+			else {
+				throw new IndexOutOfBoundsException();
+			}
+		}
+		else if (source.hasArray()) {
+			// move directly into the byte array
+			put(offset, source.array(), sourceOffset + source.arrayOffset(), numBytes);
+
+			// this must be after the get() call to ensue that the byte buffer is not
+			// modified in case the call fails
+			source.position(sourceOffset + numBytes);
+		}
+		else {
+			// neither heap buffer nor direct buffer
+			while (source.hasRemaining()) {
+				put(offset++, source.get());
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//  Utilities for native memory accesses and checks
+	// --------------------------------------------------------------------------------------------
+
+	/** The reflection fields with which we access the off-heap pointer from direct ByteBuffers */
+	private static final Field ADDRESS_FIELD;
+
+	static {
+		try {
+			ADDRESS_FIELD = java.nio.Buffer.class.getDeclaredField("address");
+			ADDRESS_FIELD.setAccessible(true);
+		}
+		catch (Throwable t) {
+			throw new RuntimeException(
+					"Cannot initialize HybridMemorySegment: off-heap memory is incompatible with this JVM.", t);
+		}
+	}
+
+	private static long getAddress(ByteBuffer buffer) {
+		if (buffer == null) {
+			throw new NullPointerException("buffer is null");
+		}
+		try {
+			return (Long) ADDRESS_FIELD.get(buffer);
+		}
+		catch (Throwable t) {
+			throw new RuntimeException("Could not access direct byte buffer address.", t);
+		}
+	}
+	
+	private static long checkBufferAndGetAddress(ByteBuffer buffer) {
+		if (buffer == null) {
+			throw new NullPointerException("buffer is null");
+		}
+		if (!buffer.isDirect()) {
+			throw new IllegalArgumentException("Can't initialize from non-direct ByteBuffer.");
+		}
+		return getAddress(buffer);
+	}
+
+	// -------------------------------------------------------------------------
+	//  Factoring
+	// -------------------------------------------------------------------------
+
+	/**
+	 * Base factory for hybrid memory segments.
+	 */
+	public static final class HybridMemorySegmentFactory implements MemorySegmentFactory.Factory {
+		
+		@Override
+		public HybridMemorySegment wrap(byte[] memory) {
+			return new HybridMemorySegment(memory);
+		}
+
+		@Override
+		public HybridMemorySegment allocateUnpooledSegment(int size, Object owner) {
+			return new HybridMemorySegment(new byte[size], owner);
+		}
+
+		@Override
+		public HybridMemorySegment wrapPooledHeapMemory(byte[] memory, Object owner) {
+			return new HybridMemorySegment(memory, owner);
+		}
+
+		@Override
+		public HybridMemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner) {
+			return new HybridMemorySegment(memory, owner);
+		}
+
+		/** prevent external instantiation */
+		HybridMemorySegmentFactory() {}
+	};
+
+	public static final HybridMemorySegmentFactory FACTORY = new HybridMemorySegmentFactory();
+}


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java
new file mode 100644
index 0000000..454c821
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java
@@ -0,0 +1,1633 @@
+/*
+ * 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.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+
+import java.nio.ByteBuffer;
+import java.util.Random;
+
+@SuppressWarnings("ConstantConditions")
+public class MemorySegmentSpeedBenchmark {
+	
+	private static final long LONG_VALUE = 0x1234567890abcdefl;
+	
+	private static final boolean TEST_CORE_ON_HEAP = true;
+	private static final boolean TEST_CORE_OFF_HEAP = false;
+	
+	// we keep this to make sure the JIT does not eliminate certain loops
+	public static long sideEffect = 0L;
+	
+	
+	public static void main(String[] args) {
+		
+		final int SMALL_SEGMENT_SIZE = 32 * 1024;
+		final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
+		
+		final int SMALL_SEGMENTS_ROUNDS = 100000;
+		final int LARGE_SEGMENT_ROUNDS = 10;
+		
+		final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
+		final byte[] smallSegment = new byte[SMALL_SEGMENT_SIZE];
+		
+		final ByteBuffer largeOffHeap = ByteBuffer.allocateDirect(LARGE_SEGMENT_SIZE);
+		final ByteBuffer smallOffHeap = ByteBuffer.allocateDirect(SMALL_SEGMENT_SIZE);
+
+		System.out.println("testing access of individual bytes");
+		
+		testPutBytes(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
+		testGetBytes(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE, SMALL_SEGMENTS_ROUNDS);
+		testPutBytes(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
+		testGetBytes(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE, LARGE_SEGMENT_ROUNDS);
+
+		System.out.println("testing access of byte arrays");
+
+		testPutByteArrays1024(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
+		testGetByteArrays1024(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 1024, SMALL_SEGMENTS_ROUNDS);
+		testPutByteArrays1024(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
+		testGetByteArrays1024(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 1024, LARGE_SEGMENT_ROUNDS);
+		
+		System.out.println("testing access of longs");
+		
+		testPutLongs(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+		testGetLongs(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+		testPutLongs(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+		testGetLongs(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+
+//		System.out.println("testing access of big endian longs");
+//		
+//		testPutLongsBigEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+//		testGetLongsBigEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+//		testPutLongsBigEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+//		testGetLongsBigEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+//
+//		System.out.println("testing access of little endian longs");
+//		
+//		testPutLongsLittleEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+//		testGetLongsLittleEndian(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 8, SMALL_SEGMENTS_ROUNDS);
+//		testPutLongsLittleEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+//		testGetLongsLittleEndian(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 8, LARGE_SEGMENT_ROUNDS);
+
+		System.out.println("testing access of ints");
+		
+		testPutInts(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
+		testGetInts(smallSegment, smallOffHeap, SMALL_SEGMENT_SIZE / 4, SMALL_SEGMENTS_ROUNDS);
+		testPutInts(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
+		testGetInts(largeSegment, largeOffHeap, LARGE_SEGMENT_SIZE / 4, LARGE_SEGMENT_ROUNDS);
+
+
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                                  BYTEs
+	// --------------------------------------------------------------------------------------------
+
+	private static void testPutBytes(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+										final int numValues, final int rounds) {
+		
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timePutBytesOnHeap(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timePutBytesHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timePutBytesHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutBytesAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutBytesAbstract(seg, numValues, rounds);
+			}
+		};
+		
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timePutBytesAbstract(seg, numValues, rounds);
+			}
+		};
+		
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Writing %d x %d bytes to %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+
+	private static void testGetBytes(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+										final int numValues, final int rounds) {
+
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timeGetBytesOnHeap(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timeGetBytesHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timeGetBytesHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetBytesAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetBytesAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timeGetBytesAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Reading %d x %d bytes from %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+
+	private static long timePutBytesOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, (byte) i);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+	private static long timePutBytesOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, (byte) i);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+	private static long timePutBytesHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, (byte) i);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+	private static long timePutBytesAbstract(final MemorySegment segment, final int num, final int rounds) {
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, (byte) i);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timeGetBytesOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.get(offset);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+
+	private static long timeGetBytesOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.get(offset);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+
+	private static long timeGetBytesHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.get(offset);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+
+
+	private static long timeGetBytesAbstract(final MemorySegment segment, final int num, final int rounds) {
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.get(offset);
+				offset++;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                                  LONGs
+	// --------------------------------------------------------------------------------------------
+	
+	private static void testPutLongs(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+										final int numValues, final int rounds) {
+
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timePutLongsOnHeap(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timePutLongsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timePutLongsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutLongsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutLongsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timePutLongsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Writing %d x %d longs to %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+	
+	private static void testGetLongs(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+										final int numValues, final int rounds) {
+
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timeGetLongsOnHeap(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timeGetLongsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timeGetLongsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetLongsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetLongsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timeGetLongsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Reading %d x %d longs from %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+	
+	private static long timePutLongsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putLong(offset, LONG_VALUE);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timePutLongsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putLong(offset, LONG_VALUE);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timePutLongsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putLong(offset, LONG_VALUE);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+	private static long timePutLongsAbstract(final MemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putLong(offset, LONG_VALUE);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timeGetLongsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getLong(offset);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+	
+	private static long timeGetLongsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getLong(offset);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+	
+	private static long timeGetLongsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getLong(offset);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+
+	private static long timeGetLongsAbstract(final MemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getLong(offset);
+				offset += 8;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                                  INTs
+	// --------------------------------------------------------------------------------------------
+	
+	private static void testPutInts(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+									final int numValues, final int rounds) {
+
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timePutIntsOnHeap(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timePutIntsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timePutIntsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutIntsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutIntsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timePutIntsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Writing %d x %d ints to %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+	
+	private static void testGetInts(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+									final int numValues, final int rounds) {
+
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timeGetIntsOnHeap(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timeGetIntsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timeGetIntsHybrid(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetIntsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetIntsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timeGetIntsAbstract(seg, numValues, rounds);
+			}
+		};
+
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Reading %d x %d ints from %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+	
+	private static long timePutIntsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putInt(offset, i);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timePutIntsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putInt(offset, i);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timePutIntsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putInt(offset, i);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+	private static long timePutIntsAbstract(final MemorySegment segment, final int num, final int rounds) {
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.putInt(offset, i);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timeGetIntsOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+		int l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getInt(offset);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+	
+	private static long timeGetIntsOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+		int l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getInt(offset);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+	
+	private static long timeGetIntsHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+		int l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getInt(offset);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+
+	private static long timeGetIntsAbstract(final MemorySegment segment, final int num, final int rounds) {
+		int l = 0;
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				l += segment.getInt(offset);
+				offset += 4;
+			}
+		}
+		long end = System.nanoTime();
+		sideEffect += l;
+		return end - start;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                                  BYTE ARRAYs
+	// --------------------------------------------------------------------------------------------
+	
+	private static void testPutByteArrays1024(final byte[] heapMemory, final ByteBuffer offHeapMemory, 
+												final int numValues, final int rounds) {
+		
+		final byte[] sourceArray = new byte[1024];
+		for (int i = 0; i < sourceArray.length; i++) {
+			sourceArray[i] = (byte) i;
+		}
+
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timePutByteArrayOnHeap(seg, sourceArray, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timePutByteArrayHybrid(seg, sourceArray, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timePutByteArrayHybrid(seg, sourceArray, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutByteArrayAbstract(seg, sourceArray, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timePutByteArrayAbstract(seg, sourceArray, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timePutByteArrayAbstract(seg, sourceArray, numValues, rounds);
+			}
+		};
+
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Writing %d x %d byte[1024] to %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+	
+	private static void testGetByteArrays1024(final byte[] heapMemory, final ByteBuffer offHeapMemory,
+												final int numValues, final int rounds) {
+		
+		final byte[] targetArray = new byte[1024];
+
+		TestRunner pureHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHeapMemorySegment seg = new PureHeapMemorySegment(heapMemory);
+				return timeGetByteArrayOnHeap(seg, targetArray, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(heapMemory);
+				return timeGetByteArrayHybrid(seg, targetArray, numValues, rounds);
+			}
+		};
+
+		TestRunner pureHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				PureHybridMemorySegment seg = new PureHybridMemorySegment(offHeapMemory);
+				return timeGetByteArrayHybrid(seg, targetArray, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetByteArrayAbstract(seg, targetArray, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOnHeap(heapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(heapMemory, null);
+				return timeGetByteArrayAbstract(seg, targetArray, numValues, rounds);
+			}
+		};
+
+		TestRunner coreHybridOffHeapRunner = new TestRunner() {
+			@Override
+			public long runTest() {
+				fillOffHeap(offHeapMemory, (byte) 0);
+				MemorySegment seg = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(offHeapMemory, null);
+				return timeGetByteArrayAbstract(seg, targetArray, numValues, rounds);
+			}
+		};
+
+		TestRunner[] tests = {
+				TEST_CORE_ON_HEAP ? coreHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridHeapRunner : null,
+				TEST_CORE_OFF_HEAP ? coreHybridOffHeapRunner : null,
+				pureHeapRunner, pureHybridHeapRunner, pureHybridOffHeapRunner
+		};
+
+		long[] results = runTestsInRandomOrder(tests, new Random(), 5, true);
+
+		System.out.println(String.format(
+				"Reading %d x %d byte[1024] from %d bytes segment: " +
+						"\n\theap=%,d msecs" +
+						"\n\thybrid-on-heap=%,d msecs" +
+						"\n\thybrid-off-heap=%,d msecs" +
+						"\n\tspecialized heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-heap=%,d msecs, " +
+						"\n\tspecialized-hybrid-off-heap=%,d msecs, ",
+				rounds, numValues, heapMemory.length,
+				(results[0] / 1000000), (results[1] / 1000000), (results[2] / 1000000),
+				(results[3] / 1000000), (results[4] / 1000000), (results[5] / 1000000)));
+	}
+	
+	private static long timePutByteArrayOnHeap(final PureHeapMemorySegment segment, final byte[] source, final int num, final int rounds) {
+		final int len = source.length;
+		
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, source, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timePutByteArrayOffHeap(final PureOffHeapMemorySegment segment, final byte[] source, final int num, final int rounds) {
+		final int len = source.length;
+		
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, source, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timePutByteArrayHybrid(final PureHybridMemorySegment segment, final byte[] source, final int num, final int rounds) {
+		final int len = source.length;
+		
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, source, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+	private static long timePutByteArrayAbstract(final MemorySegment segment, final byte[] source, final int num, final int rounds) {
+		final int len = source.length;
+
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.put(offset, source, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timeGetByteArrayOnHeap(final PureHeapMemorySegment segment, final byte[] target, final int num, final int rounds) {
+		final int len = target.length;
+		
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.get(offset, target, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timeGetByteArrayOffHeap(final PureOffHeapMemorySegment segment, final byte[] target, final int num, final int rounds) {
+		final int len = target.length;
+		
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.get(offset, target, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+	
+	private static long timeGetByteArrayHybrid(final PureHybridMemorySegment segment, final byte[] target, final int num, final int rounds) {
+		final int len = target.length;
+		
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.get(offset, target, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+	private static long timeGetByteArrayAbstract(final MemorySegment segment, final byte[] target, final int num, final int rounds) {
+		final int len = target.length;
+
+		// checked segment
+		long start = System.nanoTime();
+		for (int round = 0; round < rounds; round++) {
+			int offset = 0;
+			for (int i = 0; i < num; i++) {
+				segment.get(offset, target, 0, len);
+				offset += len;
+			}
+		}
+		long end = System.nanoTime();
+		return end - start;
+	}
+
+//	// --------------------------------------------------------------------------------------------
+//	//                                  LONG BIG ENDIAN
+//	// --------------------------------------------------------------------------------------------
+//
+//	private static void testPutLongsBigEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+//		// test the pure heap memory 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+//		long elapsedOnHeap = timePutLongsBigEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+//		// test the pure off-heap memory
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+//		long elapsedOffHeap = timePutLongsBigEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+//		// test hybrid on heap 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+//		long elapsedHybridOnHeap = timePutLongsBigEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+//		// test hybrid off heap 
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+//		long elapsedHybridOffHeap = timePutLongsBigEndianHybrid(hybridOffeap, numValues, rounds);
+//
+//		System.out.println(String.format(
+//				"Writing %d x %d big-endian longs to %d bytes segment: " +
+//						"heap=%,d msecs, " +
+//						"off-heap=%,d msecs, " +
+//						"hybrid-on-heap=%,d msecs, " +
+//						"hybrid-off-heap=%,d msecs",
+//				rounds, numValues, heapMemory.length,
+//				(elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+//				(elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+//	}
+//
+//	private static void testGetLongsBigEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+//		// test the pure heap memory 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+//		long elapsedOnHeap = timeGetLongsBigEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+//		// test the pure off-heap memory
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+//		long elapsedOffHeap = timeGetLongsBigEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+//		// test hybrid on heap 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+//		long elapsedHybridOnHeap = timeGetLongsBigEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+//		// test hybrid off heap 
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+//		long elapsedHybridOffHeap = timeGetLongsBigEndianHybrid(hybridOffeap, numValues, rounds);
+//
+//		System.out.println(String.format(
+//				"Reading %d x %d big-endian longs from %d bytes segment: " +
+//						"heap=%,d msecs, " +
+//						"off-heap=%,d msecs, " +
+//						"hybrid-on-heap=%,d msecs, " +
+//						"hybrid-off-heap=%,d msecs",
+//				rounds, numValues, heapMemory.length,
+//				(elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+//				(elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+//	}
+//
+//	private static long timePutLongsBigEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				segment.putLongBigEndian(offset, LONG_VALUE);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		return end - start;
+//	}
+//
+//	private static long timePutLongsBigEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+//		// checked segment
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				segment.putLongBigEndian(offset, LONG_VALUE);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		return end - start;
+//	}
+//
+//	private static long timePutLongsBigEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+//		// checked segment
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				segment.putLongBigEndian(offset, LONG_VALUE);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		return end - start;
+//	}
+//
+//	private static long timeGetLongsBigEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+//		long l = 0;
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				l += segment.getLongBigEndian(offset);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		sideEffect += l;
+//		return end - start;
+//	}
+//
+//	private static long timeGetLongsBigEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+//		long l = 0;
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				l += segment.getLongBigEndian(offset);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		sideEffect += l;
+//		return end - start;
+//	}
+//
+//	private static long timeGetLongsBigEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+//		// checked segment
+//		long l = 0;
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				l += segment.getLongBigEndian(offset);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		sideEffect += l;
+//		return end - start;
+//	}
+//
+//	// --------------------------------------------------------------------------------------------
+//	//                                  LONG LITTLE ENDIAN
+//	// --------------------------------------------------------------------------------------------
+//
+//	private static void testPutLongsLittleEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+//		// test the pure heap memory 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+//		long elapsedOnHeap = timePutLongsLittleEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+//		// test the pure off-heap memory
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+//		long elapsedOffHeap = timePutLongsLittleEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+//		// test hybrid on heap 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+//		long elapsedHybridOnHeap = timePutLongsLittleEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+//		// test hybrid off heap 
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+//		long elapsedHybridOffHeap = timePutLongsLittleEndianHybrid(hybridOffeap, numValues, rounds);
+//
+//		System.out.println(String.format(
+//				"Writing %d x %d little-endian longs to %d bytes segment: " +
+//						"heap=%,d msecs, " +
+//						"off-heap=%,d msecs, " +
+//						"hybrid-on-heap=%,d msecs, " +
+//						"hybrid-off-heap=%,d msecs",
+//				rounds, numValues, heapMemory.length,
+//				(elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+//				(elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+//	}
+//
+//	private static void testGetLongsLittleEndian(byte[] heapMemory, ByteBuffer offHeapMemory, int numValues, int rounds) {
+//		// test the pure heap memory 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHeapMemorySegment heapMemorySegment = new PureHeapMemorySegment(heapMemory);
+//		long elapsedOnHeap = timeGetLongsLittleEndianOnHeap(heapMemorySegment, numValues, rounds);
+//
+//		// test the pure off-heap memory
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureOffHeapMemorySegment offHeapMemorySegment = new PureOffHeapMemorySegment(offHeapMemory);
+//		long elapsedOffHeap = timeGetLongsLittleEndianOffHeap(offHeapMemorySegment, numValues, rounds);
+//
+//		// test hybrid on heap 
+//		fillOnHeap(heapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOnHeap = new PureHybridMemorySegment(heapMemory);
+//		long elapsedHybridOnHeap = timeGetLongsLittleEndianHybrid(hybridOnHeap, numValues, rounds);
+//
+//		// test hybrid off heap 
+//		fillOffHeap(offHeapMemory, (byte) 0);
+//		PureHybridMemorySegment hybridOffeap = new PureHybridMemorySegment(offHeapMemory);
+//		long elapsedHybridOffHeap = timeGetLongsLittleEndianHybrid(hybridOffeap, numValues, rounds);
+//
+//		System.out.println(String.format(
+//				"Reading %d x %d little-endian longs from %d bytes segment: " +
+//						"heap=%,d msecs, " +
+//						"off-heap=%,d msecs, " +
+//						"hybrid-on-heap=%,d msecs, " +
+//						"hybrid-off-heap=%,d msecs",
+//				rounds, numValues, heapMemory.length,
+//				(elapsedOnHeap / 1000000), (elapsedOffHeap / 1000000),
+//				(elapsedHybridOnHeap / 1000000), (elapsedHybridOffHeap / 1000000)));
+//	}
+//
+//	private static long timePutLongsLittleEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				segment.putLongLittleEndian(offset, LONG_VALUE);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		return end - start;
+//	}
+//
+//	private static long timePutLongsLittleEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+//		// checked segment
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				segment.putLongLittleEndian(offset, LONG_VALUE);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		return end - start;
+//	}
+//
+//	private static long timePutLongsLittleEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+//		// checked segment
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				segment.putLongLittleEndian(offset, LONG_VALUE);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		return end - start;
+//	}
+//
+//	private static long timeGetLongsLittleEndianOnHeap(final PureHeapMemorySegment segment, final int num, final int rounds) {
+//		long l = 0;
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				l += segment.getLongLittleEndian(offset);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		sideEffect += l;
+//		return end - start;
+//	}
+//
+//	private static long timeGetLongsLittleEndianOffHeap(final PureOffHeapMemorySegment segment, final int num, final int rounds) {
+//		long l = 0;
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				l += segment.getLongLittleEndian(offset);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		sideEffect += l;
+//		return end - start;
+//	}
+//
+//	private static long timeGetLongsLittleEndianHybrid(final PureHybridMemorySegment segment, final int num, final int rounds) {
+//		// checked segment
+//		long l = 0;
+//		long start = System.nanoTime();
+//		for (int round = 0; round < rounds; round++) {
+//			int offset = 0;
+//			for (int i = 0; i < num; i++) {
+//				l += segment.getLongLittleEndian(offset);
+//				offset += 8;
+//			}
+//		}
+//		long end = System.nanoTime();
+//		sideEffect += l;
+//		return end - start;
+//	}
+	
+	// ------------------------------------------------------------------------
+	//  Utilities
+	// ------------------------------------------------------------------------
+
+	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);
+		}
+	}
+	
+	private static long[] runTestsInRandomOrder(TestRunner[] runners, Random rnd, int numRuns, boolean printMeasures) {
+		if (numRuns < 3) {
+			throw new IllegalArgumentException("must do at least three runs");
+		}
+		
+		// we run all runners in random order, to account for the JIT effects that specialize methods
+		// The observation is that either earlier tests suffer from performance because the JIT needs to kick
+		// in first, or that later tests suffer from performance, because the HIT optimized for the other case already
+		
+		long[][] measures = new long[runners.length][];
+		for (int i = 0; i < measures.length; i++) {
+			measures[i] = new long[numRuns];
+		}
+		
+		for (int test = 0; test < numRuns; test++) {
+			System.out.println("Round " + (test+1) + '/' + numRuns);
+			
+			// pick an order for the tests
+			int[] order = new int[runners.length];
+			for (int i = 0; i < order.length; i++) {
+				order[i] = i;
+			}
+			for (int i = order.length; i > 1; i--) {
+				int pos1 = i-1;
+				int pos2 = rnd.nextInt(i);
+				int tmp = order[pos1];
+				order[pos1] = order[pos2];
+				order[pos2] = tmp;
+			}
+			
+			// run tests
+			for (int pos : order) {
+				TestRunner next = runners[pos];
+				measures[pos][test] = next != null ? next.runTest() : 0L;
+			}
+		}
+		
+		if (printMeasures) {
+			for (long[] series : measures) {
+				StringBuilder bld = new StringBuilder();
+				for (long measure : series) {
+					bld.append(String.format("%,d", (measure / 1000000))).append(" | ");
+				}
+				System.out.println(bld.toString());
+			}
+		}
+		
+		// aggregate the measures
+		long[] results = new long[runners.length];
+		
+		for (int i = 0; i < runners.length; i++) {
+			// cancel out the min and max
+			long max = Long.MIN_VALUE;
+			long min = Long.MAX_VALUE;
+			
+			for (long val : measures[i]) {
+				max = Math.max(max, val);
+				min = Math.min(min, val);
+			}
+			
+			long total = 0L;
+			for (long val : measures[i]) {
+				if (val != max && val != min) {
+					total += val;
+				}
+			}
+			
+			results[i] = total / (numRuns - 2);
+		}
+		
+		return results;
+	}
+	
+	
+	
+	private static interface TestRunner {
+		
+		long runTest();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java
new file mode 100644
index 0000000..e247eed
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java
@@ -0,0 +1,466 @@
+/*
+ * 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.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public final class PureHeapMemorySegment {
+
+	/** Constant that flags the byte order. Because this is a boolean constant,
+	 * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+	private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+
+	/** The array in which the data is stored. */
+	private byte[] memory;
+
+	/** Wrapper for I/O requests. */
+	private ByteBuffer wrapper;
+
+	/** The size, stored extra, because we may clear the reference to the byte array */
+	private final int size;
+
+	// -------------------------------------------------------------------------
+	//                             Constructors
+	// -------------------------------------------------------------------------
+
+	/**
+	 * Creates a new memory segment that represents the data in the given byte array.
+	 *
+	 * @param memory The byte array that holds the data.
+	 */
+	public PureHeapMemorySegment(byte[] memory) {
+		this.memory = memory;
+		this.size = memory.length;
+	}
+
+	// -------------------------------------------------------------------------
+	//                      Direct Memory Segment Specifics
+	// -------------------------------------------------------------------------
+
+	/**
+	 * Gets the byte array that backs this memory segment.
+	 *
+	 * @return The byte array that backs this memory segment.
+	 */
+	public byte[] getArray() {
+		return this.memory;
+	}
+
+	// -------------------------------------------------------------------------
+	//                        MemorySegment Accessors
+	// -------------------------------------------------------------------------
+	
+	public final boolean isFreed() {
+		return this.memory == null;
+	}
+	
+	public final void free() {
+		this.wrapper = null;
+		this.memory = null;
+	}
+	
+	public final int size() {
+		return this.size;
+	}
+	
+	public final ByteBuffer wrap(int offset, int length) {
+		if (offset > this.memory.length || offset > this.memory.length - length) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		if (this.wrapper == null) {
+			this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
+		}
+		else {
+			this.wrapper.limit(offset + length);
+			this.wrapper.position(offset);
+		}
+
+		return this.wrapper;
+	}
+	
+	// ------------------------------------------------------------------------
+	//                    Random Access get() and put() methods
+	// ------------------------------------------------------------------------
+	
+	public final byte get(int index) {
+		return this.memory[index];
+	}
+	
+	public final void put(int index, byte b) {
+		this.memory[index] = b;
+	}
+	
+	public final void get(int index, byte[] dst) {
+		get(index, dst, 0, dst.length);
+	}
+	
+	public final void put(int index, byte[] src) {
+		put(index, src, 0, src.length);
+	}
+	
+	public final void get(int index, byte[] dst, int offset, int length) {
+		// system arraycopy does the boundary checks anyways, no need to check extra
+		System.arraycopy(this.memory, index, dst, offset, length);
+	}
+	
+	public final void put(int index, byte[] src, int offset, int length) {
+		// system arraycopy does the boundary checks anyways, no need to check extra
+		System.arraycopy(src, offset, this.memory, index, length);
+	}
+	
+	public final boolean getBoolean(int index) {
+		return this.memory[index] != 0;
+	}
+	
+	public final void putBoolean(int index, boolean value) {
+		this.memory[index] = (byte) (value ? 1 : 0);
+	}
+	
+	@SuppressWarnings("restriction")
+	public final char getChar(int index) {
+		if (index >= 0 && index <= this.memory.length - 2) {
+			return UNSAFE.getChar(this.memory, BASE_OFFSET + index);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final char getCharLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getChar(index);
+		} else {
+			return Character.reverseBytes(getChar(index));
+		}
+	}
+
+	public final char getCharBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Character.reverseBytes(getChar(index));
+		} else {
+			return getChar(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putChar(int index, char value) {
+		if (index >= 0 && index <= this.memory.length - 2) {
+			UNSAFE.putChar(this.memory, BASE_OFFSET + index, value);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putCharLittleEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, value);
+		} else {
+			putChar(index, Character.reverseBytes(value));
+		}
+	}
+
+	public final void putCharBigEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, Character.reverseBytes(value));
+		} else {
+			putChar(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final short getShort(int index) {
+		if (index >= 0 && index <= this.memory.length - 2) {
+			return UNSAFE.getShort(this.memory, BASE_OFFSET + index);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final short getShortLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getShort(index);
+		} else {
+			return Short.reverseBytes(getShort(index));
+		}
+	}
+	
+	public final short getShortBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Short.reverseBytes(getShort(index));
+		} else {
+			return getShort(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putShort(int index, short value) {
+		if (index >= 0 && index <= this.memory.length - 2) {
+			UNSAFE.putShort(this.memory, BASE_OFFSET + index, value);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putShortLittleEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, value);
+		} else {
+			putShort(index, Short.reverseBytes(value));
+		}
+	}
+	
+	public final void putShortBigEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, Short.reverseBytes(value));
+		} else {
+			putShort(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final int getInt(int index) {
+		if (index >= 0 && index <= this.memory.length - 4) {
+			return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final int getIntLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getInt(index);
+		} else {
+			return Integer.reverseBytes(getInt(index));
+		}
+	}
+	
+	public final int getIntBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Integer.reverseBytes(getInt(index));
+		} else {
+			return getInt(index);
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void putInt(int index, int value) {
+		if (index >= 0 && index <= this.memory.length - 4) {
+			UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putIntLittleEndian(int index, int value) {
+		if (LITTLE_ENDIAN) {
+			putInt(index, value);
+		} else {
+			putInt(index, Integer.reverseBytes(value));
+		}
+	}
+	
+	public final void putIntBigEndian(int index, int value) {
+		if (LITTLE_ENDIAN) {
+			putInt(index, Integer.reverseBytes(value));
+		} else {
+			putInt(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final long getLong(int index) {
+		if (index >= 0 && index <= this.memory.length - 8) {
+			return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final long getLongLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getLong(index);
+		} else {
+			return Long.reverseBytes(getLong(index));
+		}
+	}
+	
+	public final long getLongBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Long.reverseBytes(getLong(index));
+		} else {
+			return getLong(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putLong(int index, long value) {
+		if (index >= 0 && index <= this.memory.length - 8) {
+			UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
+		} else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putLongLittleEndian(int index, long value) {
+		if (LITTLE_ENDIAN) {
+			putLong(index, value);
+		} else {
+			putLong(index, Long.reverseBytes(value));
+		}
+	}
+	
+	public final void putLongBigEndian(int index, long value) {
+		if (LITTLE_ENDIAN) {
+			putLong(index, Long.reverseBytes(value));
+		} else {
+			putLong(index, value);
+		}
+	}
+
+	public final float getFloat(int index) {
+		return Float.intBitsToFloat(getInt(index));
+	}
+	
+	public final float getFloatLittleEndian(int index) {
+		return Float.intBitsToFloat(getIntLittleEndian(index));
+	}
+	
+	public final float getFloatBigEndian(int index) {
+		return Float.intBitsToFloat(getIntBigEndian(index));
+	}
+	
+	public final void putFloat(int index, float value) {
+		putInt(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final void putFloatLittleEndian(int index, float value) {
+		putIntLittleEndian(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final void putFloatBigEndian(int index, float value) {
+		putIntBigEndian(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final double getDouble(int index) {
+		return Double.longBitsToDouble(getLong(index));
+	}
+	
+	public final double getDoubleLittleEndian(int index) {
+		return Double.longBitsToDouble(getLongLittleEndian(index));
+	}
+
+	public final double getDoubleBigEndian(int index) {
+		return Double.longBitsToDouble(getLongBigEndian(index));
+	}
+	
+	public final void putDouble(int index, double value) {
+		putLong(index, Double.doubleToRawLongBits(value));
+	}
+	
+	public final void putDoubleLittleEndian(int index, double value) {
+		putLongLittleEndian(index, Double.doubleToRawLongBits(value));
+	}
+	
+	public final void putDoubleBigEndian(int index, double value) {
+		putLongBigEndian(index, Double.doubleToRawLongBits(value));
+	}
+
+	// -------------------------------------------------------------------------
+	//                     Bulk Read and Write Methods
+	// -------------------------------------------------------------------------
+	
+	public final void get(DataOutput out, int offset, int length) throws IOException {
+		out.write(this.memory, offset, length);
+	}
+	
+	public final void put(DataInput in, int offset, int length) throws IOException {
+		in.readFully(this.memory, offset, length);
+	}
+	
+	public final void get(int offset, ByteBuffer target, int numBytes) {
+		// ByteBuffer performs the boundary checks
+		target.put(this.memory, offset, numBytes);
+	}
+	
+	public final void put(int offset, ByteBuffer source, int numBytes) {
+		// ByteBuffer performs the boundary checks
+		source.get(this.memory, offset, numBytes);
+	}
+	
+	public final void copyTo(int offset, PureHeapMemorySegment target, int targetOffset, int numBytes) {
+		// system arraycopy does the boundary checks anyways, no need to check extra
+		System.arraycopy(this.memory, offset, target.memory, targetOffset, numBytes);
+	}
+
+	// -------------------------------------------------------------------------
+	//                      Comparisons & Swapping
+	// -------------------------------------------------------------------------
+	
+	public final int compare(PureHeapMemorySegment seg2, int offset1, int offset2, int len) {
+		final byte[] b2 = seg2.memory;
+		final byte[] b1 = this.memory;
+
+		int val = 0;
+		for (int pos = 0; pos < len && (val = (b1[offset1 + pos] & 0xff) - (b2[offset2 + pos] & 0xff)) == 0; pos++);
+		return val;
+	}
+
+	public final void swapBytes(PureHeapMemorySegment seg2, int offset1, int offset2, int len) {
+		// swap by bytes (chunks of 8 first, then single bytes)
+		while (len >= 8) {
+			long tmp = this.getLong(offset1);
+			this.putLong(offset1, seg2.getLong(offset2));
+			seg2.putLong(offset2, tmp);
+			offset1 += 8;
+			offset2 += 8;
+			len -= 8;
+		}
+		while (len > 0) {
+			byte tmp = this.get(offset1);
+			this.put(offset1, seg2.get(offset2));
+			seg2.put(offset2, tmp);
+			offset1++;
+			offset2++;
+			len--;
+		}
+	}
+	
+	public final void swapBytes(byte[] auxBuffer, PureHeapMemorySegment seg2, int offset1, int offset2, int len) {
+		byte[] otherMem = seg2.memory;
+		System.arraycopy(this.memory, offset1, auxBuffer, 0, len);
+		System.arraycopy(otherMem, offset2, this.memory, offset1, len);
+		System.arraycopy(auxBuffer, 0, otherMem, offset2, len);
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                     Utilities for native memory accesses and checks
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("restriction")
+	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+	@SuppressWarnings("restriction")
+	private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java
new file mode 100644
index 0000000..1e3b89e
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java
@@ -0,0 +1,359 @@
+/*
+ * 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 java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class PureHeapMemorySegmentOutView implements DataOutputView {
+
+	private PureHeapMemorySegment 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<PureHeapMemorySegment> memorySource;
+	
+	private final List<PureHeapMemorySegment> fullSegments;
+	
+
+	private byte[] utfBuffer;		// the reusable array for UTF encodings
+
+
+	public PureHeapMemorySegmentOutView(List<PureHeapMemorySegment> emptySegments,
+										List<PureHeapMemorySegment> 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 PureHeapMemorySegment nextSegment(PureHeapMemorySegment current, int positionInCurrent) throws EOFException {
+		int size = this.memorySource.size();
+		if (size > 0) {
+			final PureHeapMemorySegment next = this.memorySource.remove(size - 1);
+			this.fullSegments.add(next);
+			return next;
+		} else {
+			throw new EOFException();
+		}
+	}
+	
+	public PureHeapMemorySegment 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(PureHeapMemorySegment 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();
+		}
+	}
+}


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java
new file mode 100644
index 0000000..4c92234
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/EndiannessAccessChecks.java
@@ -0,0 +1,183 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import java.util.Random;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class EndiannessAccessChecks {
+	
+	@Test
+	public void testHeapSegment() {
+		try {
+			testBigAndLittleEndianAccessUnaligned(new HeapMemorySegment(new byte[11111]));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testHybridOnHeapSegment() {
+		try {
+			testBigAndLittleEndianAccessUnaligned(new HybridMemorySegment(new byte[11111]));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testHybridOffHeapSegment() {
+		try {
+			testBigAndLittleEndianAccessUnaligned(new HybridMemorySegment(ByteBuffer.allocateDirect(11111)));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private void testBigAndLittleEndianAccessUnaligned(MemorySegment segment) {
+		final Random rnd = new Random();
+		
+		// longs
+		{
+			final long seed = rnd.nextLong();
+			
+			rnd.setSeed(seed);
+			for (int i = 0; i < 10000; i++) {
+				long val = rnd.nextLong();
+				int pos = rnd.nextInt(segment.size - 7);
+				
+				segment.putLongLittleEndian(pos, val);
+				long r = segment.getLongBigEndian(pos);
+				assertEquals(val, Long.reverseBytes(r));
+
+				segment.putLongBigEndian(pos, val);
+				r = segment.getLongLittleEndian(pos);
+				assertEquals(val, Long.reverseBytes(r));
+			}
+		}
+
+		// ints
+		{
+			final long seed = rnd.nextLong();
+
+			rnd.setSeed(seed);
+			for (int i = 0; i < 10000; i++) {
+				int val = rnd.nextInt();
+				int pos = rnd.nextInt(segment.size - 3);
+
+				segment.putIntLittleEndian(pos, val);
+				int r = segment.getIntBigEndian(pos);
+				assertEquals(val, Integer.reverseBytes(r));
+
+				segment.putIntBigEndian(pos, val);
+				r = segment.getIntLittleEndian(pos);
+				assertEquals(val, Integer.reverseBytes(r));
+			}
+		}
+
+		// shorts
+		{
+			final long seed = rnd.nextLong();
+
+			rnd.setSeed(seed);
+			for (int i = 0; i < 10000; i++) {
+				short val = (short) rnd.nextInt();
+				int pos = rnd.nextInt(segment.size - 1);
+
+				segment.putShortLittleEndian(pos, val);
+				short r = segment.getShortBigEndian(pos);
+				assertEquals(val, Short.reverseBytes(r));
+
+				segment.putShortBigEndian(pos, val);
+				r = segment.getShortLittleEndian(pos);
+				assertEquals(val, Short.reverseBytes(r));
+			}
+		}
+
+		// chars
+		{
+			final long seed = rnd.nextLong();
+
+			rnd.setSeed(seed);
+			for (int i = 0; i < 10000; i++) {
+				char val = (char) rnd.nextInt();
+				int pos = rnd.nextInt(segment.size - 1);
+
+				segment.putCharLittleEndian(pos, val);
+				char r = segment.getCharBigEndian(pos);
+				assertEquals(val, Character.reverseBytes(r));
+
+				segment.putCharBigEndian(pos, val);
+				r = segment.getCharLittleEndian(pos);
+				assertEquals(val, Character.reverseBytes(r));
+			}
+		}
+
+		// floats
+		{
+			final long seed = rnd.nextLong();
+
+			rnd.setSeed(seed);
+			for (int i = 0; i < 10000; i++) {
+				float val = rnd.nextFloat();
+				int pos = rnd.nextInt(segment.size - 3);
+
+				segment.putFloatLittleEndian(pos, val);
+				float r = segment.getFloatBigEndian(pos);
+				float reversed = Float.intBitsToFloat(Integer.reverseBytes(Float.floatToRawIntBits(r)));
+				assertEquals(val, reversed, 0.0f);
+
+				segment.putFloatBigEndian(pos, val);
+				r = segment.getFloatLittleEndian(pos);
+				reversed = Float.intBitsToFloat(Integer.reverseBytes(Float.floatToRawIntBits(r)));
+				assertEquals(val, reversed, 0.0f);
+			}
+		}
+
+		// doubles
+		{
+			final long seed = rnd.nextLong();
+
+			rnd.setSeed(seed);
+			for (int i = 0; i < 10000; i++) {
+				double val = rnd.nextDouble();
+				int pos = rnd.nextInt(segment.size - 7);
+
+				segment.putDoubleLittleEndian(pos, val);
+				double r = segment.getDoubleBigEndian(pos);
+				double reversed = Double.longBitsToDouble(Long.reverseBytes(Double.doubleToRawLongBits(r)));
+				assertEquals(val, reversed, 0.0f);
+
+				segment.putDoubleBigEndian(pos, val);
+				r = segment.getDoubleLittleEndian(pos);
+				reversed = Double.longBitsToDouble(Long.reverseBytes(Double.doubleToRawLongBits(r)));
+				assertEquals(val, reversed, 0.0f);
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java
new file mode 100644
index 0000000..d7a5b03
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/HeapMemorySegmentTest.java
@@ -0,0 +1,71 @@
+/*
+ * 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 org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.*;
+
+@RunWith(Parameterized.class)
+public class HeapMemorySegmentTest extends MemorySegmentTestBase {
+
+	public HeapMemorySegmentTest(int pageSize) {
+		super(pageSize);
+	}
+
+	@Override
+	MemorySegment createSegment(int size) {
+		return new HeapMemorySegment(new byte[size]);
+	}
+
+	@Override
+	MemorySegment createSegment(int size, Object owner) {
+		return new HeapMemorySegment(new byte[size], owner);
+	}
+	
+	@Test
+	public void testHeapSegmentSpecifics() {
+		try {
+			final byte[] buffer = new byte[411];
+			HeapMemorySegment seg = new HeapMemorySegment(buffer);
+			
+			assertFalse(seg.isFreed());
+			assertFalse(seg.isOffHeap());
+			assertEquals(buffer.length, seg.size());
+			assertTrue(buffer == seg.getArray());
+
+			ByteBuffer buf1 = seg.wrap(1, 2);
+			ByteBuffer buf2 = seg.wrap(3, 4);
+
+			assertTrue(buf1 != buf2);
+			assertEquals(1, buf1.position());
+			assertEquals(3, buf1.limit());
+			assertEquals(3, buf2.position());
+			assertEquals(7, buf2.limit());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java
new file mode 100644
index 0000000..b09697e
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOffHeapMemorySegmentTest.java
@@ -0,0 +1,84 @@
+/*
+ * 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 org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@RunWith(Parameterized.class)
+public class HybridOffHeapMemorySegmentTest extends MemorySegmentTestBase {
+
+	public HybridOffHeapMemorySegmentTest(int pageSize) {
+		super(pageSize);
+	}
+
+	@Override
+	MemorySegment createSegment(int size) {
+		return new HybridMemorySegment(ByteBuffer.allocateDirect(size));
+	}
+
+	@Override
+	MemorySegment createSegment(int size, Object owner) {
+		return new HybridMemorySegment(ByteBuffer.allocateDirect(size), owner);
+	}
+
+	@Test
+	public void testHybridHeapSegmentSpecifics() {
+		try {
+			final ByteBuffer buffer = ByteBuffer.allocateDirect(411);
+			HybridMemorySegment seg = new HybridMemorySegment(buffer);
+
+			assertFalse(seg.isFreed());
+			assertTrue(seg.isOffHeap());
+			assertEquals(buffer.capacity(), seg.size());
+			assertTrue(buffer == seg.getOffHeapBuffer());
+
+			try {
+				seg.getArray();
+				fail("should throw an exception");
+			}
+			catch (IllegalStateException e) {
+				// expected
+			}
+
+			ByteBuffer buf1 = seg.wrap(1, 2);
+			ByteBuffer buf2 = seg.wrap(3, 4);
+
+			assertTrue(buf1 != buffer);
+			assertTrue(buf2 != buffer);
+			assertTrue(buf1 != buf2);
+			assertEquals(1, buf1.position());
+			assertEquals(3, buf1.limit());
+			assertEquals(3, buf2.position());
+			assertEquals(7, buf2.limit());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java
new file mode 100644
index 0000000..55d333e
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/HybridOnHeapMemorySegmentTest.java
@@ -0,0 +1,82 @@
+/*
+ * 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 org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+@RunWith(Parameterized.class)
+public class HybridOnHeapMemorySegmentTest extends MemorySegmentTestBase {
+
+	public HybridOnHeapMemorySegmentTest(int pageSize) {
+		super(pageSize);
+	}
+
+	@Override
+	MemorySegment createSegment(int size) {
+		return new HybridMemorySegment(new byte[size]);
+	}
+
+	@Override
+	MemorySegment createSegment(int size, Object owner) {
+		return new HybridMemorySegment(new byte[size], owner);
+	}
+	
+	@Test
+	public void testHybridHeapSegmentSpecifics() {
+		try {
+			final byte[] buffer = new byte[411];
+			HybridMemorySegment seg = new HybridMemorySegment(buffer);
+
+			assertFalse(seg.isFreed());
+			assertFalse(seg.isOffHeap());
+			assertEquals(buffer.length, seg.size());
+			assertTrue(buffer == seg.getArray());
+			
+			try {
+				seg.getOffHeapBuffer();
+				fail("should throw an exception");
+			}
+			catch (IllegalStateException e) {
+				// expected
+			}
+			
+			ByteBuffer buf1 = seg.wrap(1, 2);
+			ByteBuffer buf2 = seg.wrap(3, 4);
+			
+			assertTrue(buf1 != buf2);
+			assertEquals(1, buf1.position());
+			assertEquals(3, buf1.limit());
+			assertEquals(3, buf2.position());
+			assertEquals(7, buf2.limit());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java
new file mode 100644
index 0000000..f50322c
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentChecksTest.java
@@ -0,0 +1,135 @@
+/*
+ * 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.DataInput;
+import java.io.DataOutput;
+import java.nio.ByteBuffer;
+
+/**
+ * Tests for the sanity checks of the memory segments.
+ */
+public class MemorySegmentChecksTest {
+	
+	@Test(expected = NullPointerException.class)
+	public void testHeapNullBuffer1() {
+		new HeapMemorySegment(null);
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void testHeapNullBuffer2() {
+		new HeapMemorySegment(null, new Object());
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void testHybridHeapNullBuffer1() {
+		new HybridMemorySegment((byte[]) null);
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void testHybridHeapNullBuffer2() {
+		new HybridMemorySegment((byte[]) null, new Object());
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void testHybridOffHeapNullBuffer1() {
+		new HybridMemorySegment((ByteBuffer) null);
+	}
+
+	@Test(expected = NullPointerException.class)
+	public void testHybridOffHeapNullBuffer2() {
+		new HybridMemorySegment((ByteBuffer) null, new Object());
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testHybridNonDirectBuffer() {
+		new HybridMemorySegment(ByteBuffer.allocate(1024));
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testZeroAddress(){
+		new MockSegment(0L, 4*1024, null);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testNegativeAddress(){
+		new MockSegment(-1L, 4*1024, null);
+	}
+
+	@Test(expected = IllegalArgumentException.class)
+	public void testTooLargeAddress(){
+		new MockSegment(Long.MAX_VALUE - 8*1024, 4*1024, null);
+	}
+	
+	// ------------------------------------------------------------------------
+	
+	final class MockSegment extends MemorySegment {
+
+		MockSegment(long offHeapAddress, int size, Object owner) {
+			super(offHeapAddress, size, owner);
+		}
+
+		@Override
+		public ByteBuffer wrap(int offset, int length) {
+			return null;
+		}
+
+		@Override
+		public byte get(int index) {
+			return 0;
+		}
+
+		@Override
+		public void put(int index, byte b) {}
+
+		@Override
+		public void get(int index, byte[] dst) {}
+
+		@Override
+		public void put(int index, byte[] src) {}
+
+		@Override
+		public void get(int index, byte[] dst, int offset, int length) {}
+
+		@Override
+		public void put(int index, byte[] src, int offset, int length) {}
+
+		@Override
+		public boolean getBoolean(int index) {
+			return false;
+		}
+
+		@Override
+		public void putBoolean(int index, boolean value) {}
+
+		@Override
+		public void get(DataOutput out, int offset, int length) {}
+
+		@Override
+		public void put(DataInput in, int offset, int length) {}
+
+		@Override
+		public void get(int offset, ByteBuffer target, int numBytes) {}
+
+		@Override
+		public void put(int offset, ByteBuffer source, int numBytes) {}
+	};
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java
new file mode 100644
index 0000000..10c3622
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/MemorySegmentTestBase.java
@@ -0,0 +1,2571 @@
+/*
+ * 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.Assert;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Random;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for the access and transfer methods of the HeapMemorySegment.
+ */
+public abstract class MemorySegmentTestBase {
+	
+	private final Random random = new Random();
+	
+	private final int pageSize;
+	
+	
+	public MemorySegmentTestBase(int pageSize) {
+		this.pageSize = pageSize;
+	}
+
+	// ------------------------------------------------------------------------
+	//  Access to primitives
+	// ------------------------------------------------------------------------
+
+	abstract MemorySegment createSegment(int size);
+
+	abstract MemorySegment createSegment(int size, Object owner);
+	
+	// ------------------------------------------------------------------------
+	//  Access to primitives
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testByteAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+			try {
+				segment.put(-1, (byte) 0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(pageSize, (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(pageSize);
+				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);
+			}
+
+			// test expected correct behavior, sequential access
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i < pageSize; i++) {
+				segment.put(i, (byte) random.nextInt());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i < pageSize; i++) {
+				assertEquals((byte) random.nextInt(), segment.get(i));
+			}
+
+			// test expected correct behavior, random access
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize);
+
+				if (occupied[pos]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+				}
+
+				segment.put(pos, (byte) random.nextInt());
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize);
+
+				if (occupied[pos]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+				}
+
+				assertEquals((byte) random.nextInt(), segment.get(pos));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+
+	}
+
+	@Test
+	public void testBooleanAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+			try {
+				segment.putBoolean(-1, false);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putBoolean(pageSize, false);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putBoolean(Integer.MAX_VALUE, false);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putBoolean(Integer.MIN_VALUE, false);
+				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(pageSize);
+				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);
+			}
+
+			// test expected correct behavior, sequential access
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i < pageSize; i++) {
+				segment.putBoolean(i, random.nextBoolean());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i < pageSize; i++) {
+				assertEquals(random.nextBoolean(), segment.getBoolean(i));
+			}
+
+			// test expected correct behavior, random access
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize);
+
+				if (occupied[pos]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+				}
+
+				segment.putBoolean(pos, random.nextBoolean());
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize);
+
+				if (occupied[pos]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+				}
+
+				assertEquals(random.nextBoolean(), segment.getBoolean(pos));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testCharAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+
+			try {
+				segment.putChar(-1, 'a');
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putChar(pageSize, '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.putChar(Integer.MAX_VALUE, 'a');
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putChar(Integer.MAX_VALUE - 1, 'a');
+				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(pageSize);
+				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);
+			}
+
+			try {
+				segment.getChar(Integer.MAX_VALUE);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getChar(Integer.MAX_VALUE - 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// test expected correct behavior, sequential access 
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 2; i += 2) {
+				segment.putChar(i, (char) (random.nextInt(Character.MAX_VALUE)));
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 2; i += 2) {
+				assertEquals((char) (random.nextInt(Character.MAX_VALUE)), segment.getChar(i));
+			}
+
+			// test expected correct behavior, random access 
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 1);
+
+				if (occupied[pos] || occupied[pos + 1]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+				}
+
+				segment.putChar(pos, (char) (random.nextInt(Character.MAX_VALUE)));
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 1);
+
+				if (occupied[pos] || occupied[pos + 1]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+				}
+
+				assertEquals((char) (random.nextInt(Character.MAX_VALUE)), segment.getChar(pos));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testShortAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+
+			try {
+				segment.putShort(-1, (short) 0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putShort(pageSize, (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.putShort(Integer.MAX_VALUE, (short) 0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putShort(Integer.MAX_VALUE - 1, (short) 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(pageSize);
+				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);
+			}
+
+			try {
+				segment.getShort(Integer.MAX_VALUE);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getShort(Integer.MAX_VALUE - 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// test expected correct behavior, sequential access
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 2; i += 2) {
+				segment.putShort(i, (short) random.nextInt());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 2; i += 2) {
+				assertEquals((short) random.nextInt(), segment.getShort(i));
+			}
+
+			// test expected correct behavior, random access
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 1);
+
+				if (occupied[pos] || occupied[pos + 1]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+				}
+
+				segment.putShort(pos, (short) random.nextInt());
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 1);
+
+				if (occupied[pos] || occupied[pos + 1]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+				}
+
+				assertEquals((short) random.nextInt(), segment.getShort(pos));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testIntAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+
+			try {
+				segment.putInt(-1, 0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putInt(pageSize, 0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putInt(pageSize - 3, 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.putInt(Integer.MAX_VALUE, 0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putInt(Integer.MAX_VALUE - 3, 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(pageSize);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getInt(pageSize - 3);
+				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);
+			}
+
+			try {
+				segment.getInt(Integer.MAX_VALUE);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getInt(Integer.MAX_VALUE - 3);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// test expected correct behavior, sequential access
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 4; i += 4) {
+				segment.putInt(i, random.nextInt());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 4; i += 4) {
+				assertEquals(random.nextInt(), segment.getInt(i));
+			}
+
+			// test expected correct behavior, random access
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 3);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+				}
+
+				segment.putInt(pos, random.nextInt());
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 3);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+				}
+
+				assertEquals(random.nextInt(), segment.getInt(pos));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testLongAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+
+			try {
+				segment.putLong(-1, 0L);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putLong(pageSize, 0L);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putLong(pageSize - 7, 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.putLong(Integer.MAX_VALUE, 0L);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putLong(Integer.MAX_VALUE - 7, 0L);
+				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(pageSize);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getLong(pageSize - 7);
+				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);
+			}
+
+			try {
+				segment.getLong(Integer.MAX_VALUE);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getLong(Integer.MAX_VALUE - 7);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// test expected correct behavior, sequential access
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 8; i += 8) {
+				segment.putLong(i, random.nextLong());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 8; i += 8) {
+				assertEquals(random.nextLong(), segment.getLong(i));
+			}
+
+			// test expected correct behavior, random access
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 7);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+						occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+				{
+					continue;
+				}
+				else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+					occupied[pos+4] = true;
+					occupied[pos+5] = true;
+					occupied[pos+6] = true;
+					occupied[pos+7] = true;
+				}
+
+				segment.putLong(pos, random.nextLong());
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 7);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+						occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+				{
+					continue;
+				}
+				else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+					occupied[pos+4] = true;
+					occupied[pos+5] = true;
+					occupied[pos+6] = true;
+					occupied[pos+7] = true;
+				}
+
+				assertEquals(random.nextLong(), segment.getLong(pos));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testFloatAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+
+			try {
+				segment.putFloat(-1, 0.0f);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putFloat(pageSize, 0.0f);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putFloat(pageSize - 3, 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.putFloat(Integer.MAX_VALUE, 0.0f);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putFloat(Integer.MAX_VALUE - 3, 0.0f);
+				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(pageSize);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getFloat(pageSize - 3);
+				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);
+			}
+
+			try {
+				segment.getFloat(Integer.MAX_VALUE);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getFloat(Integer.MAX_VALUE - 3);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// test expected correct behavior, sequential access
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 4; i += 4) {
+				segment.putFloat(i, random.nextFloat());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 4; i += 4) {
+				assertEquals(random.nextFloat(), segment.getFloat(i), 0.0);
+			}
+
+			// test expected correct behavior, random access
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 3);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+				}
+
+				segment.putFloat(pos, random.nextFloat());
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 3);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3]) {
+					continue;
+				} else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+				}
+
+				assertEquals(random.nextFloat(), segment.getFloat(pos), 0.0);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testDoubleAccess() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			// test exceptions
+			try {
+				segment.putDouble(-1, 0.0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putDouble(pageSize, 0.0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putDouble(pageSize - 7, 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.putDouble(Integer.MAX_VALUE, 0.0);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.putDouble(Integer.MAX_VALUE - 7, 0.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(pageSize);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getDouble(pageSize - 7);
+				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);
+			}
+
+			try {
+				segment.getDouble(Integer.MAX_VALUE);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.getDouble(Integer.MAX_VALUE - 7);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// test expected correct behavior, sequential access
+
+			long seed = random.nextLong();
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 8; i += 8) {
+				segment.putDouble(i, random.nextDouble());
+			}
+
+			random.setSeed(seed);
+			for (int i = 0; i <= pageSize - 8; i += 8) {
+				assertEquals(random.nextDouble(), segment.getDouble(i), 0.0);
+			}
+
+			// test expected correct behavior, random access
+
+			random.setSeed(seed);
+			boolean[] occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 7);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+						occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+				{
+					continue;
+				}
+				else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+					occupied[pos+4] = true;
+					occupied[pos+5] = true;
+					occupied[pos+6] = true;
+					occupied[pos+7] = true;
+				}
+
+				segment.putDouble(pos, random.nextDouble());
+			}
+
+			random.setSeed(seed);
+			occupied = new boolean[pageSize];
+
+			for (int i = 0; i < 1000; i++) {
+				int pos = random.nextInt(pageSize - 7);
+
+				if (occupied[pos] || occupied[pos + 1] || occupied[pos + 2] || occupied[pos + 3] ||
+						occupied[pos + 4] || occupied[pos + 5] || occupied[pos + 6] || occupied[pos + 7])
+				{
+					continue;
+				}
+				else {
+					occupied[pos] = true;
+					occupied[pos+1] = true;
+					occupied[pos+2] = true;
+					occupied[pos+3] = true;
+					occupied[pos+4] = true;
+					occupied[pos+5] = true;
+					occupied[pos+6] = true;
+					occupied[pos+7] = true;
+				}
+
+				assertEquals(random.nextDouble(), segment.getDouble(pos), 0.0);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Bulk Byte Movements
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testBulkBytePutExceptions() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			byte[] bytes = new byte[pageSize / 4 + (pageSize%4)];
+			random.nextBytes(bytes);
+
+			// wrong positions into memory segment
+
+			try {
+				segment.put(-1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(-1, bytes, 4, 5);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(Integer.MIN_VALUE, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(Integer.MIN_VALUE, bytes, 4, 5);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(pageSize, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(pageSize, bytes, 6, 44);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(pageSize - bytes.length + 1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(pageSize - 5, bytes, 3, 6);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(Integer.MAX_VALUE, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(Integer.MAX_VALUE, bytes, 10, 20);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(Integer.MAX_VALUE - bytes.length + 1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(Integer.MAX_VALUE - 11, bytes, 11, 11);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(3 * (pageSize / 4) + 1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(3 * (pageSize / 4) + 2, bytes, 0, bytes.length - 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(7 * (pageSize / 8) + 1, bytes, 0, bytes.length / 2);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// wrong source array positions / lengths
+
+			try {
+				segment.put(0, bytes, -1, 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(0, bytes, -1, bytes.length + 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(0, bytes, Integer.MIN_VALUE, bytes.length);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(0, bytes, Integer.MAX_VALUE, bytes.length);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.put(0, bytes, Integer.MAX_VALUE - bytes.length + 1, bytes.length);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// case where negative offset and negative index compensate each other
+			try {
+				segment.put(-2, bytes, -1, bytes.length / 2);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testBulkByteGetExceptions() {
+		try {
+			final MemorySegment segment = createSegment(pageSize);
+
+			byte[] bytes = new byte[pageSize / 4];
+
+			// wrong positions into memory segment
+
+			try {
+				segment.get(-1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(-1, bytes, 4, 5);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(Integer.MIN_VALUE, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(Integer.MIN_VALUE, bytes, 4, 5);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(pageSize, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(pageSize, bytes, 6, 44);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(pageSize - bytes.length + 1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(pageSize - 5, bytes, 3, 6);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(Integer.MAX_VALUE, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(Integer.MAX_VALUE, bytes, 10, 20);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(Integer.MAX_VALUE - bytes.length + 1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(Integer.MAX_VALUE - 11, bytes, 11, 11);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(3 * (pageSize / 4) + 1, bytes);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(3 * (pageSize / 4) + 2, bytes, 0, bytes.length - 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(7 * (pageSize / 8) + 1, bytes, 0, bytes.length / 2);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// wrong source array positions / lengths
+
+			try {
+				segment.get(0, bytes, -1, 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(0, bytes, -1, bytes.length + 1);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(0, bytes, Integer.MIN_VALUE, bytes.length);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(0, bytes, Integer.MAX_VALUE, bytes.length);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			try {
+				segment.get(0, bytes, Integer.MAX_VALUE - bytes.length + 1, bytes.length);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+
+			// case where negative offset and negative index compensate each other
+			try {
+				segment.get(-2, bytes, -1, bytes.length / 2);
+				fail("IndexOutOfBoundsException expected");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IndexOutOfBoundsException);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testBulkByteAccess() {
+		try {
+			// test expected correct behavior with default offset / length
+			{
+				final MemorySegment segment = createSegment(pageSize);
+				long seed = random.nextLong();
+
+				random.setSeed(seed);
+				byte[] src = new byte[pageSize / 8];
+				for (int i = 0; i < 8; i++) {
+					random.nextBytes(src);
+					segment.put(i * (pageSize / 8), src);
+				}
+
+				random.setSeed(seed);
+				byte[] expected = new byte[pageSize / 8];
+				byte[] actual = new byte[pageSize / 8];
+				for (int i = 0; i < 8; i++) {
+					random.nextBytes(expected);
+					segment.get(i * (pageSize / 8), actual);
+
+					assertArrayEquals(expected, actual);
+				}
+			}
+
+			// test expected correct behavior with specific offset / length
+			{
+				final MemorySegment segment = createSegment(pageSize);
+				byte[] expected = new byte[pageSize];
+				random.nextBytes(expected);
+
+				for (int i = 0; i < 16; i++) {
+					segment.put(i * (pageSize / 16), expected, i * (pageSize / 16),
+							pageSize / 16);
+				}
+
+				byte[] actual = new byte[pageSize];
+				for (int i = 0; i < 16; i++) {
+					segment.get(i * (pageSize / 16), actual, i * (pageSize / 16),
+							pageSize / 16);
+				}
+
+				assertArrayEquals(expected, actual);
+			}
+			
+			// put segments of various lengths to various positions
+			{
+				final MemorySegment segment = createSegment(pageSize);
+				byte[] expected = new byte[pageSize];
+				
+				for (int i = 0; i < 200; i++) {
+					int numBytes = random.nextInt(pageSize - 10) + 1;
+					int pos = random.nextInt(pageSize - numBytes + 1);
+					
+					byte[] data = new byte[(random.nextInt(3) + 1) * numBytes];
+					int dataStartPos = random.nextInt(data.length - numBytes + 1);
+					
+					random.nextBytes(data);
+					
+					// copy to the expected
+					System.arraycopy(data, dataStartPos, expected, pos, numBytes);
+					
+					// put to the memory segment
+					segment.put(pos, data, dataStartPos, numBytes);
+				}
+				
+				byte[] validation = new byte[pageSize];
+				segment.get(0, validation);
+				
+				assertArrayEquals(expected, validation);
+			}
+			
+			// get segments with various contents
+			{
+				final MemorySegment segment = createSegment(pageSize);
+				byte[] contents = new byte[pageSize];
+				random.nextBytes(contents);
+				segment.put(0, contents);
+
+				for (int i = 0; i < 200; i++) {
+					int numBytes = random.nextInt(pageSize / 8) + 1;
+					int pos = random.nextInt(pageSize - numBytes + 1);
+
+					byte[] data = new byte[(random.nextInt(3) + 1) * numBytes];
+					int dataStartPos = random.nextInt(data.length - numBytes + 1);
+
+					segment.get(pos, data, dataStartPos, numBytes);
+					
+					byte[] expected = Arrays.copyOfRange(contents, pos, pos + numBytes);
+					byte[] validation = Arrays.copyOfRange(data, dataStartPos, dataStartPos + numBytes);
+					assertArrayEquals(expected, validation);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Writing / Reading to/from DataInput / DataOutput
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testDataInputOutput() {
+		try {
+			MemorySegment seg = createSegment(pageSize);
+			byte[] contents = new byte[pageSize];
+			random.nextBytes(contents);
+			seg.put(0, contents);
+
+			ByteArrayOutputStream buffer = new ByteArrayOutputStream(pageSize);
+			DataOutputStream out = new DataOutputStream(buffer);
+
+			// write the segment in chunks into the stream
+			int pos = 0;
+			while (pos < pageSize) {
+				int len = random.nextInt(200);
+				len = Math.min(len, pageSize - pos);
+				seg.get(out, pos, len);
+				pos += len;
+			}
+
+			// verify that we wrote the same bytes
+			byte[] result = buffer.toByteArray();
+			assertArrayEquals(contents, result);
+
+			// re-read the bytes into a new memory segment
+			MemorySegment reader = createSegment(pageSize);
+			DataInputStream in = new DataInputStream(new ByteArrayInputStream(result));
+
+			pos = 0;
+			while (pos < pageSize) {
+				int len = random.nextInt(200);
+				len = Math.min(len, pageSize - pos);
+				reader.put(in, pos, len);
+				pos += len;
+			}
+
+			byte[] targetBuffer = new byte[pageSize];
+			reader.get(0, targetBuffer);
+			
+			assertArrayEquals(contents, targetBuffer);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testDataInputOutputOutOfBounds() {
+		try {
+			final int segmentSize = 52;
+			
+			// segment with random contents
+			MemorySegment seg = createSegment(segmentSize);
+			byte[] bytes = new byte[segmentSize];
+			random.nextBytes(bytes);
+			seg.put(0, bytes);
+			
+			// out of bounds when writing
+			{
+				DataOutputStream out = new DataOutputStream(new ByteArrayOutputStream());
+					
+				try {
+					seg.get(out, -1, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+	
+				try {
+					seg.get(out, segmentSize, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+	
+				try {
+					seg.get(out, -segmentSize, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+	
+				try {
+					seg.get(out, Integer.MIN_VALUE, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+	
+				try {
+					seg.get(out, Integer.MAX_VALUE, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+			}
+
+			// out of bounds when reading
+			{
+				DataInputStream in = new DataInputStream(new ByteArrayInputStream(new byte[segmentSize]));
+
+				try {
+					seg.put(in, -1, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+
+				try {
+					seg.put(in, segmentSize, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+
+				try {
+					seg.put(in, -segmentSize, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+
+				try {
+					seg.put(in, Integer.MIN_VALUE, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+
+				try {
+					seg.put(in, Integer.MAX_VALUE, segmentSize / 2);
+					fail("IndexOutOfBoundsException expected");
+				}
+				catch (Exception e) {
+					assertTrue(e instanceof IndexOutOfBoundsException);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testDataInputOutputStreamUnderflowOverflow() {
+		try {
+			final int segmentSize = 1337;
+
+			// segment with random contents
+			MemorySegment seg = createSegment(segmentSize);
+			byte[] bytes = new byte[segmentSize];
+			random.nextBytes(bytes);
+			seg.put(0, bytes);
+			
+			// a stream that we cannot fully write to
+			DataOutputStream out = new DataOutputStream(new OutputStream() {
+				
+				int bytesSoFar = 0;
+				@Override
+				public void write(int b) throws IOException {
+					bytesSoFar++;
+					if (bytesSoFar > segmentSize / 2) {
+						throw new IOException("overflow");
+					}
+				}
+			});
+
+			// write the segment in chunks into the stream
+			try {
+				int pos = 0;
+				while (pos < pageSize) {
+					int len = random.nextInt(segmentSize / 10);
+					len = Math.min(len, pageSize - pos);
+					seg.get(out, pos, len);
+					pos += len;
+				}
+				fail("Should fail with an IOException");
+			}
+			catch (IOException e) {
+				// expected
+			}
+			
+			DataInputStream in = new DataInputStream(new ByteArrayInputStream(new byte[segmentSize / 2]));
+
+			try {
+				int pos = 0;
+				while (pos < pageSize) {
+					int len = random.nextInt(segmentSize / 10);
+					len = Math.min(len, pageSize - pos);
+					seg.put(in, pos, len);
+					pos += len;
+				}
+				fail("Should fail with an EOFException");
+			}
+			catch (EOFException e) {
+				// expected
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  ByteBuffer Ops
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testByteBufferGet() {
+		try {
+			testByteBufferGet(false);
+			testByteBufferGet(true);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private void testByteBufferGet(boolean directBuffer) {
+		MemorySegment seg = createSegment(pageSize);
+		byte[] bytes = new byte[pageSize];
+		random.nextBytes(bytes);
+		seg.put(0, bytes);
+
+		ByteBuffer target = directBuffer ?
+				ByteBuffer.allocateDirect(3 * pageSize) :
+				ByteBuffer.allocate(3 * pageSize);
+		target.position(2 * pageSize);
+
+		// transfer the segment in chunks into the byte buffer
+		int pos = 0;
+		while (pos < pageSize) {
+			int len = random.nextInt(pageSize / 10);
+			len = Math.min(len, pageSize - pos);
+			seg.get(pos, target, len);
+			pos += len;
+		}
+
+		// verify that we wrote the same bytes
+		byte[] result = new byte[pageSize];
+		target.position(2 * pageSize);
+		target.get(result);
+
+		assertArrayEquals(bytes, result);
+	}
+
+	@Test
+	public void testByteBufferPut() {
+		try {
+			testByteBufferPut(false);
+			testByteBufferPut(true);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	private void testByteBufferPut(boolean directBuffer) {
+		byte[] bytes = new byte[pageSize];
+		random.nextBytes(bytes);
+
+		ByteBuffer source = directBuffer ?
+			ByteBuffer.allocateDirect(pageSize) :
+			ByteBuffer.allocate(pageSize);
+		
+		source.put(bytes);
+		source.clear();
+
+		MemorySegment seg = createSegment(3 * pageSize);
+
+		int offset = 2 * pageSize;
+
+		// transfer the segment in chunks into the byte buffer
+		int pos = 0;
+		while (pos < pageSize) {
+			int len = random.nextInt(pageSize / 10);
+			len = Math.min(len, pageSize - pos);
+			seg.put(offset + pos, source, len);
+			pos += len;
+		}
+
+		// verify that we read the same bytes
+		byte[] result = new byte[pageSize];
+		seg.get(offset, result);
+
+		assertArrayEquals(bytes, result);
+	}
+
+	// ------------------------------------------------------------------------
+	//  ByteBuffer Ops on sliced byte buffers
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testSlicedByteBufferGet() {
+		try {
+			testSlicedByteBufferGet(false);
+			testSlicedByteBufferGet(true);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	private void testSlicedByteBufferGet(boolean directBuffer) {
+		MemorySegment seg = createSegment(pageSize);
+		byte[] bytes = new byte[pageSize];
+		random.nextBytes(bytes);
+		seg.put(0, bytes);
+
+		ByteBuffer target = directBuffer ?
+				ByteBuffer.allocateDirect(pageSize + 49) :
+				ByteBuffer.allocate(pageSize + 49);
+		
+		target.position(19).limit(19 + pageSize);
+		
+		ByteBuffer slicedTarget = target.slice();
+
+		// transfer the segment in chunks into the byte buffer
+		int pos = 0;
+		while (pos < pageSize) {
+			int len = random.nextInt(pageSize / 10);
+			len = Math.min(len, pageSize - pos);
+			seg.get(pos, slicedTarget, len);
+			pos += len;
+		}
+
+		// verify that we wrote the same bytes
+		byte[] result = new byte[pageSize];
+		target.position(19);
+		target.get(result);
+
+		assertArrayEquals(bytes, result);
+	}
+
+	@Test
+	public void testSlicedByteBufferPut() {
+		try {
+			testSlicedByteBufferPut(false);
+			testSlicedByteBufferPut(true);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	private void testSlicedByteBufferPut(boolean directBuffer) {
+		byte[] bytes = new byte[pageSize + 49];
+		random.nextBytes(bytes);
+
+		ByteBuffer source = directBuffer ?
+				ByteBuffer.allocateDirect(pageSize + 49) :
+				ByteBuffer.allocate(pageSize + 49);
+		
+		source.put(bytes);
+		source.position(19).limit(19 + pageSize);
+		ByteBuffer slicedSource = source.slice();
+
+		MemorySegment seg = createSegment(3 * pageSize);
+
+		final int offset = 2 * pageSize;
+
+		// transfer the segment in chunks into the byte buffer
+		int pos = 0;
+		while (pos < pageSize) {
+			int len = random.nextInt(pageSize / 10);
+			len = Math.min(len, pageSize - pos);
+			seg.put(offset + pos, slicedSource, len);
+			pos += len;
+		}
+
+		// verify that we read the same bytes
+		byte[] result = new byte[pageSize];
+		seg.get(offset, result);
+
+		byte[] expected = Arrays.copyOfRange(bytes, 19, 19 + pageSize);
+		assertArrayEquals(expected, result);
+	}
+
+	// ------------------------------------------------------------------------
+	//  ByteBuffer overflow / underflow and out of bounds
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testByteBufferOutOfBounds() {
+		try {
+			final int bbCapacity = pageSize / 10;
+			
+			final int[] validOffsets = { 0, 1, pageSize / 10 * 9 };
+			final int[] invalidOffsets = { -1, pageSize + 1, -pageSize, Integer.MAX_VALUE, Integer.MIN_VALUE };
+			
+			final int[] validLengths = { 0, 1, bbCapacity, pageSize };
+			final int[] invalidLengths = { -1, -pageSize, Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+			final MemorySegment seg = createSegment(pageSize);
+
+			for (ByteBuffer bb : new ByteBuffer[] {
+							ByteBuffer.allocate(bbCapacity), 
+							ByteBuffer.allocateDirect(bbCapacity) } )
+			{
+				for (int off : validOffsets) {
+					for (int len : invalidLengths) {
+						try {
+							seg.put(off, bb, len);
+							fail("should fail with an IndexOutOfBoundsException");
+						}
+						catch (IndexOutOfBoundsException | BufferUnderflowException ignored) {}
+						
+						try {
+							seg.get(off, bb, len);
+							fail("should fail with an IndexOutOfBoundsException");
+						}
+						catch (IndexOutOfBoundsException | BufferOverflowException ignored) {}
+						
+						// position/limit may not have changed
+						assertEquals(0, bb.position());
+						assertEquals(bb.capacity(), bb.limit());
+					}
+				}
+	
+				for (int off : invalidOffsets) {
+					for (int len : validLengths) {
+						try {
+							seg.put(off, bb, len);
+							fail("should fail with an IndexOutOfBoundsException");
+						}
+						catch (IndexOutOfBoundsException | BufferUnderflowException ignored) {}
+	
+						try {
+							seg.get(off, bb, len);
+							fail("should fail with an IndexOutOfBoundsException");
+						}
+						catch (IndexOutOfBoundsException | BufferOverflowException ignored) {}
+	
+						// position/limit may not have changed
+						assertEquals(0, bb.position());
+						assertEquals(bb.capacity(), bb.limit());
+					}
+				}
+	
+				for (int off : validOffsets) {
+					for (int len : validLengths) {
+						if (off + len > pageSize) {
+							try {
+								seg.put(off, bb, len);
+								fail("should fail with an IndexOutOfBoundsException");
+							}
+							catch (IndexOutOfBoundsException | BufferUnderflowException ignored) {}
+		
+							try {
+								seg.get(off, bb, len);
+								fail("should fail with an IndexOutOfBoundsException");
+							}
+							catch (IndexOutOfBoundsException | BufferOverflowException ignored) {}
+		
+							// position/limit may not have changed
+							assertEquals(0, bb.position());
+							assertEquals(bb.capacity(), bb.limit());
+						}
+					}
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testByteBufferOverflowUnderflow() {
+		try {
+			final int bbCapacity = pageSize / 10;
+			ByteBuffer bb = ByteBuffer.allocate(bbCapacity);
+			
+			MemorySegment seg = createSegment(pageSize);
+			
+			try {
+				seg.get(pageSize / 5, bb, pageSize / 10 + 2);
+				fail("should fail with an exception");
+			}
+			catch (BufferOverflowException ignored) {}
+			
+			// position / limit should not have been modified
+			assertEquals(0, bb.position());
+			assertEquals(bb.capacity(), bb.limit());
+
+			try {
+				seg.put(pageSize / 5, bb, pageSize / 10 + 2);
+				fail("should fail with an exception");
+			}
+			catch (BufferUnderflowException ignored) {}
+
+			// position / limit should not have been modified
+			assertEquals(0, bb.position());
+			assertEquals(bb.capacity(), bb.limit());
+			
+
+			int pos = bb.capacity() / 3;
+			int limit = 2 * bb.capacity() / 3;
+			bb.limit(limit);
+			bb.position(pos);
+			
+			try {
+				seg.get(20, bb, bb.capacity() / 3 + 3);
+				fail("should fail with an exception");
+			}
+			catch (BufferOverflowException ignored) {}
+
+			// position / limit should not have been modified
+			assertEquals(pos, bb.position());
+			assertEquals(limit, bb.limit());
+
+			try {
+				seg.put(20, bb, bb.capacity() / 3 + 3);
+				fail("should fail with an exception");
+			}
+			catch (BufferUnderflowException ignored) {}
+
+			// position / limit should not have been modified
+			assertEquals(pos, bb.position());
+			assertEquals(limit, bb.limit());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Comparing and swapping
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testCompareBytes() {
+		try {
+			final byte[] bytes1 = new byte[pageSize];
+			final byte[] bytes2 = new byte[pageSize];
+
+			final int stride = pageSize / 255;
+			final int shift = 16666;
+
+			for (int i = 0; i < pageSize; i++) {
+				byte val = (byte) ((i / stride) & 0xff);
+				bytes1[i] = val;
+
+				if (i + shift < bytes2.length) {
+					bytes2[i + shift] = val;
+				}
+			}
+
+			MemorySegment seg1 = createSegment(pageSize);
+			MemorySegment seg2 = createSegment(pageSize);
+			seg1.put(0, bytes1);
+			seg2.put(0, bytes2);
+
+			for (int i = 0; i < 1000; i++) {
+				int pos1 = random.nextInt(bytes1.length);
+				int pos2 = random.nextInt(bytes2.length);
+
+				int len = Math.min(Math.min(bytes1.length - pos1, bytes2.length - pos2),
+						random.nextInt(pageSize / 50 ));
+
+				int cmp = seg1.compare(seg2, pos1, pos2, len);
+
+				if (pos1 < pos2 - shift) {
+					assertTrue(cmp <= 0);
+				}
+				else {
+					assertTrue(cmp >= 0);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSwapBytes() {
+		try {
+			final int HALF_SIZE = pageSize / 2;
+
+			final byte[] bytes1 = new byte[pageSize];
+			final byte[] bytes2 = new byte[HALF_SIZE];
+
+			Arrays.fill(bytes2, (byte) 1);
+
+			MemorySegment seg1 = createSegment(pageSize);
+			MemorySegment seg2 = createSegment(HALF_SIZE);
+			seg1.put(0, bytes1);
+			seg2.put(0, bytes2);
+
+			// wap the second half of the first segment with the second segment
+
+			int pos = 0;
+			while (pos < HALF_SIZE) {
+				int len = random.nextInt(pageSize / 40);
+				len = Math.min(len, HALF_SIZE - pos);
+				seg1.swapBytes(new byte[len], seg2, pos + HALF_SIZE, pos, len);
+				pos += len;
+			}
+
+			// the second segment should now be all zeros, the first segment should have one in its second half
+
+			for (int i = 0; i < HALF_SIZE; i++) {
+				assertEquals((byte) 0, seg1.get(i));
+				assertEquals((byte) 0, seg2.get(i));
+				assertEquals((byte) 1, seg1.get(i + HALF_SIZE));
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testCheckAgainstOverflowUnderflowOnRelease() {
+		try {
+			MemorySegment seg = createSegment(512);
+			seg.free();
+			
+			// --- bytes (smallest type) --- 
+			try {
+				seg.get(0);
+				fail("Expecting an IllegalStateException");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+			}
+
+			try {
+				seg.get(Integer.MAX_VALUE);
+				fail("Expecting an IllegalStateException");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+			}
+
+			try {
+				seg.get(Integer.MIN_VALUE);
+				fail("Expecting an IllegalStateException");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+			}
+
+			// --- longs (largest type) ---
+			try {
+				seg.getLong(0);
+				fail("Expecting an IllegalStateException");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+			}
+
+			try {
+				seg.getLong(Integer.MAX_VALUE);
+				fail("Expecting an IllegalStateException");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+			}
+
+			try {
+				seg.getLong(Integer.MIN_VALUE);
+				fail("Expecting an IllegalStateException");
+			}
+			catch (Exception e) {
+				assertTrue(e instanceof IllegalStateException || e instanceof NullPointerException);
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Miscellaneous 
+	// ------------------------------------------------------------------------
+
+	@Test
+	public void testByteBufferWrapping() {
+		try {
+			MemorySegment seg = createSegment(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());
+
+			// verify that operations on the byte buffer are correctly reflected
+			// in the memory segment
+			buf3.order(ByteOrder.LITTLE_ENDIAN);
+			buf3.putInt(112, 651797651);
+			assertEquals(651797651, seg.getIntLittleEndian(112));
+
+			buf3.order(ByteOrder.BIG_ENDIAN);
+			buf3.putInt(187, 992288337);
+			assertEquals(992288337, seg.getIntBigEndian(187));
+			
+			try {
+				seg.wrap(-1, 20);
+				fail("should throw an exception");
+			}
+			catch (IndexOutOfBoundsException | IllegalArgumentException ignored) {}
+
+			try {
+				seg.wrap(10, -20);
+				fail("should throw an exception");
+			}
+			catch (IndexOutOfBoundsException | IllegalArgumentException ignored) {}
+			
+			try {
+				seg.wrap(10, 1024);
+				fail("should throw an exception");
+			}
+			catch (IndexOutOfBoundsException | IllegalArgumentException ignored) {}
+			
+			// after freeing, no wrapping should be possible any more.
+			seg.free();
+			
+			try {
+				seg.wrap(13, 47);
+				fail("should fail with an exception");
+			}
+			catch (IllegalStateException e) {
+				// expected
+			}
+			
+			// existing wraps should stay valid after freeing
+			buf3.order(ByteOrder.LITTLE_ENDIAN);
+			buf3.putInt(112, 651797651);
+			assertEquals(651797651, buf3.getInt(112));
+			buf3.order(ByteOrder.BIG_ENDIAN);
+			buf3.putInt(187, 992288337);
+			assertEquals(992288337, buf3.getInt(187));
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			Assert.fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testOwner() {
+		try {
+			// a segment without an owner has a null owner
+			assertNull(createSegment(64).getOwner());
+
+			Object theOwner = new Object();
+			MemorySegment seg = createSegment(64, theOwner);
+			assertEquals(theOwner, seg.getOwner());
+
+			// freeing must release the owner, to prevent leaks that prevent class unloading!
+			seg.free();
+			assertNotNull(seg.getOwner());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	@Test
+	public void testSizeAndFreeing() {
+		try {
+			// a segment without an owner has a null owner
+			final int SIZE = 651;
+			MemorySegment seg = createSegment(SIZE);
+
+			assertEquals(SIZE, seg.size());
+			assertFalse(seg.isFreed());
+
+			seg.free();
+			assertTrue(seg.isFreed());
+			assertEquals(SIZE, seg.size());
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	// ------------------------------------------------------------------------
+	//  Parametrization to run with different segment sizes
+	// ------------------------------------------------------------------------
+	
+	@Parameterized.Parameters(name = "segment-size = {0}")
+	public static Collection<Object[]> executionModes(){
+		return Arrays.asList(
+				new Object[] { 32*1024 },
+				new Object[] { 4*1024 },
+				new Object[] { 512*1024 } );
+	}
+}
\ No newline at end of file


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/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 ce080d3..31d5563 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
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.core.memory;
 
 import java.io.DataInput;
@@ -26,26 +25,39 @@ import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
 /**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- * 
+ * This class represents a piece of memory managed by Flink.
+ * The segment may be backed by heap memory (byte array) or by off-heap memory.
+ * <p>
+ * The methods for individual memory access are specialized in the classes
+ * {@link org.apache.flink.core.memory.HeapMemorySegment} and
+ * {@link org.apache.flink.core.memory.HybridMemorySegment}.
+ * All methods that operate across two memory segments are implemented in this class,
+ * to transparently handle the mixing of memory segment types.
  * <p>
+ * This class fulfills conceptually a similar purpose as Java's {@link java.nio.ByteBuffer}.
+ * We add this specialized class for various reasons:
+ * <ul>
+ *     <li>It offers additional binary compare, swap, and copy methods.</li>
+ *     <li>It uses collapsed checks for range check and memory segment disposal.</li>
+ *     <li>It offers absolute positioning methods for bulk put/get methods, to guarantee
+ *         thread safe use.</li>
+ *     <li>It offers explicit big-endian / little-endian access methods, rather than tracking internally
+ *         a byte order.</li>
+ *     <li>It transparently and efficiently moves data between on-heap and off-heap variants.</li>
+ * </ul>
  * 
- * Comments on the implementation: We make heavy use of operations that are supported by native
+ * <i>Comments on the implementation</i>:
+ * We make heavy use of operations that are supported by native
  * instructions, to achieve a high efficiency. Multi byte types (int, long, float, double, ...)
- * are read and written with "unsafe" native commands. Little-endian to big-endian conversion and
- * vice versa are done using the static <i>reverseBytes</i> methods in the boxing data types
- * (for example {@link Integer#reverseBytes(int)}). On x86/amd64, these are translated by the
- * jit compiler to <i>bswap</i> intrinsic commands.
- * 
- * Below is an example of the code generated for the {@link MemorySegment#putLongBigEndian(int, long)}
- * function by the just-in-time compiler. The code is grabbed from an oracle jvm 7 using the
+ * are read and written with "unsafe" native commands.
+ * <p>
+ * Below is an example of the code generated for the {@link HeapMemorySegment#putLongBigEndian(int, long)}
+ * function by the just-in-time compiler. The code is grabbed from an Oracle JVM 7 using the
  * hotspot disassembler library (hsdis32.dll) and the jvm command
- * <i>-XX:+UnlockDiagnosticVMOptions -XX:CompileCommand=print,*UnsafeMemorySegment.putLongBigEndian</i>.
+ * <i>-XX:+UnlockDiagnosticVMOptions -XX:CompileCommand=print,*MemorySegment.putLongBigEndian</i>.
  * Note that this code realizes both the byte order swapping and the reinterpret cast access to
  * get a long from the byte array.
- * 
+ *
  * <pre>
  * [Verified Entry Point]
  *   0x00007fc403e19920: sub    $0x18,%rsp
@@ -66,286 +78,539 @@ import java.nio.ByteOrder;
  *                                                 ;   {poll_return}
  *   0x00007fc403e1994a: retq 
  * </pre>
+ *
+ * <i>Note on efficiency</i>:
+ * For best efficiency, the code that uses this class should make sure that only one
+ * subclass is loaded, or that the methods that are abstract in this class are used only from one of the
+ * subclasses (either the {@link org.apache.flink.core.memory.HeapMemorySegment}, or the 
+ * {@link org.apache.flink.core.memory.HybridMemorySegment}).
+ * 
+ * That way, all the abstract methods in the MemorySegment base class have only one loaded
+ * actual implementation. This is easy for the JIT to recognize through class hierarchy analysis,
+ * or by identifying that the invocations are monomorphic (all go to the same concrete
+ * method implementation). Under these conditions, the JIT can perfectly inline methods.
  */
-public class MemorySegment {
+public abstract class MemorySegment {
+
+	/** The unsafe handle for transparent memory copied (heap / off-heap) */
+	@SuppressWarnings("restriction")
+	protected static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+	/** The beginning of the byte array contents, relative to the byte array object */
+	@SuppressWarnings("restriction")
+	protected static final long BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+	
+	/** Constant that flags the byte order. Because this is a boolean constant,
+	 * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+	private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+	
+	// ------------------------------------------------------------------------
+
+	/** The heap byte array object relative to which we access the memory. Is non-null if the
+	 *  memory is on the heap, and is null, if the memory if off the heap. If we have this buffer, we
+	 *  must never void this reference, or the memory segment will point to undefined addresses 
+	 *  outside the heap and may in out-of-order execution cases cause segmentation faults. */
+	protected final byte[] heapMemory;
+
+	/** The address to the data, relative to the heap memory byte array. If the heap memory byte array
+	 * is null, this becomes an absolute memory address outside the heap. */
+	protected long address;
+
+	/** The address one byte after the last addressable byte.
+	 *  This is address + size while the segment is not disposed */
+	protected final long addressLimit;
 	
-	// flag to enable / disable boundary checks. Note that the compiler eliminates the
-	// code paths of the checks (as dead code) when this constant is set to false.
-	private static final boolean CHECKED = true;
+	/** The size in bytes of the memory segment */
+	protected final int size;
 	
+	/** Optional owner of the memory segment */
+	private final Object owner;
+
 	/**
-	 * The array in which the data is stored.
+	 * Creates a new memory segment that represents the memory of the byte array.
+	 * Since the byte array is backed by on-heap memory, this memory segment holds its
+	 * data on heap. The buffer must be at least of size 8 bytes.
+	 *
+	 * @param buffer The byte array whose memory is represented by this memory segment.
 	 */
-	protected byte[] memory;
-	
+	MemorySegment(byte[] buffer, Object owner) {
+		if (buffer == null) {
+			throw new NullPointerException("buffer");
+		}
+		
+		this.heapMemory = buffer;
+		this.address = BYTE_ARRAY_BASE_OFFSET;
+		this.size = buffer.length;
+		this.addressLimit = this.address + this.size;
+		this.owner = owner;
+	}
+
 	/**
-	 * Wrapper for I/O requests.
+	 * Creates a new memory segment that represents the memory at the absolute address given
+	 * by the pointer.
+	 *
+	 * @param offHeapAddress The address of the memory represented by this memory segment.
+	 * @param size The size of this memory segment.
 	 */
-	protected ByteBuffer wrapper;
+	MemorySegment(long offHeapAddress, int size, Object owner) {
+		if (offHeapAddress <= 0) {
+			throw new IllegalArgumentException("negative pointer or size");
+		}
+		if (offHeapAddress >= Long.MAX_VALUE - Integer.MAX_VALUE) {
+			// this is necessary to make sure the collapsed checks are safe against numeric overflows
+			throw new IllegalArgumentException("Segment initialized with too large address: " + address
+					+ " ; Max allowed address is " + (Long.MAX_VALUE - Integer.MAX_VALUE - 1));
+		}
+		
+		this.heapMemory = null;
+		this.address = offHeapAddress;
+		this.addressLimit = this.address + size;
+		this.size = size;
+		this.owner = owner;
+	}
 	
-	// -------------------------------------------------------------------------
-	//                             Constructors
-	// -------------------------------------------------------------------------
+	// ------------------------------------------------------------------------
+	// Memory Segment Operations
+	// ------------------------------------------------------------------------
 
 	/**
-	 * Creates a new memory segment that represents the data in the given byte array.
-	 * 
-	 * @param memory The byte array that holds the data.
+	 * Gets the size of the memory segment, in bytes.
+	 * @return The size of the memory segment.
 	 */
-	public MemorySegment(byte[] memory) {
-		this.memory = memory;
+	public int size() {
+		return size;
 	}
 
-	// -------------------------------------------------------------------------
-	//                        MemorySegment Accessors
-	// -------------------------------------------------------------------------
-	
 	/**
-	 * Checks whether this memory segment has already been freed. In that case, the
-	 * segment must not be used any more.
-	 * 
-	 * @return True, if the segment has been freed, false otherwise.
+	 * Checks whether the memory segment was freed.
+	 * @return True, if the memory segment has been freed, false otherwise.
 	 */
-	public final boolean isFreed() {
-		return this.memory == null;
+	public boolean isFreed() {
+		return address > addressLimit;
 	}
 
-	public final void free() {
-		this.wrapper = null;
-		this.memory = null;
+	/**
+	 * Frees this memory segment. After this operation has been called, no further operations are
+	 * possible on the memory segment and will fail. The actual memory (heap or off-heap) will only
+	 * be released after this memory segment object has become garbage collected. 
+	 */
+	public void free() {
+		// this ensures we can place no more data and trigger
+		// the checks for the freed segment
+		address = addressLimit + 1;
 	}
-	
+
 	/**
-	 * Gets the size of the memory segment, in bytes. Because segments
-	 * are backed by arrays, they cannot be larger than two GiBytes.
-	 * 
-	 * @return The size in bytes.
+	 * Checks whether this memory segment is backed by off-heap memory.
+	 * @return True, if the memory segment is backed by off-heap memory, false if it is backed
+	 *         by heap memory.
 	 */
-	public final int size() {
-		return this.memory.length;
+	public boolean isOffHeap() {
+		return heapMemory == null;
 	}
 
 	/**
 	 * Wraps the chunk of the underlying memory located between <tt>offset<tt> and 
 	 * <tt>length</tt> in a NIO ByteBuffer.
-	 * 
+	 *
 	 * @param offset The offset in the memory segment.
 	 * @param length The number of bytes to be wrapped as a buffer.
 	 * @return A <tt>ByteBuffer</tt> backed by the specified portion of the memory segment.
 	 * @throws IndexOutOfBoundsException Thrown, if offset is negative or larger than the memory segment size,
 	 *                                   or if the offset plus the length is larger than the segment size.
 	 */
-	public ByteBuffer wrap(int offset, int length) {
-		if (offset > this.memory.length || offset > this.memory.length - length) {
-			throw new IndexOutOfBoundsException();
-		}
-		
-		if (this.wrapper == null) {
-			this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
-		}
-		else {
-			this.wrapper.limit(offset + length);
-			this.wrapper.position(offset);
-		}
+	public abstract ByteBuffer wrap(int offset, int length);
 
-		return this.wrapper;
+	/**
+	 * Gets the owner of this memory segment. Returns null, if the owner was not set.
+	 * @return The owner of the memory segment, or null, if it does not have an owner.
+	 */
+	public Object getOwner() {
+		return owner;
 	}
-
+	
+	
 	// ------------------------------------------------------------------------
 	//                    Random Access get() and put() methods
 	// ------------------------------------------------------------------------
 
-	// --------------------------------------------------------------------------------------------
-	// WARNING: Any code for range checking must take care to avoid integer overflows. The position
-	// integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
-	// <code>position + 3 &lt; end</code> may fail because <code>position + 3</code> becomes negative.
-	// A safe solution is to subtract the delta from the limit, for example
-	// <code>position &lt; end - 3</code>. Since all indices are always positive, and the integer domain
-	// has one more negative value than positive values, this can never cause an underflow.
-	// --------------------------------------------------------------------------------------------
-
-
+	//------------------------------------------------------------------------
+	// Notes on the implementation: We try to collapse as many checks as
+	// possible. We need to obey the following rules to make this safe
+	// against segfaults:
+	// 
+	//  - Grab mutable fields onto the stack before checking and using. This
+	//    guards us against concurrent modifications which invalidate the
+	//    pointers
+	//  - Use subtrations for range checks, as they are tolerant 
+	//------------------------------------------------------------------------
+	
 	/**
 	 * Reads the byte at the given position.
-	 * 
+	 *
 	 * @param index The position from which the byte will be read
 	 * @return The byte at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger or equal to the size of
 	 *                                   the memory segment.
 	 */
-	public final byte get(int index) {
-		return this.memory[index];
-	}
+	public abstract byte get(int index);
 
 	/**
 	 * Writes the given byte into this buffer at the given position.
-	 * 
+	 *
 	 * @param index The index at which the byte will be written.
 	 * @param b The byte value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger or equal to the size of
 	 *                                   the memory segment.
 	 */
-	public final void put(int index, byte b) {
-		this.memory[index] = b;
-	}
+	public abstract void put(int index, byte b);
 
 	/**
 	 * Bulk get method. Copies dst.length memory from the specified position to
 	 * the destination memory.
-	 * 
+	 *
 	 * @param index The position at which the first byte will be read.
 	 * @param dst The memory into which the memory will be copied.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large that the data between the 
 	 *                                   index and the memory segment end is not enough to fill the destination array.
 	 */
-	public final void get(int index, byte[] dst) {
-		get(index, dst, 0, dst.length);
-	}
+	public abstract void get(int index, byte[] dst);
 
 	/**
 	 * Bulk put method. Copies src.length memory from the source memory into the
 	 * memory segment beginning at the specified position.
-	 * 
+	 *
 	 * @param index The index in the memory segment array, where the data is put.
 	 * @param src The source array to copy the data from.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large such that the array 
 	 *                                   size exceed the amount of memory between the index and the memory
 	 *                                   segment's end. 
 	 */
-	public final void put(int index, byte[] src) {
-		put(index, src, 0, src.length);
-	}
+	public abstract void put(int index, byte[] src);
 
 	/**
 	 * Bulk get method. Copies length memory from the specified position to the
 	 * destination memory, beginning at the given offset
-	 * 
+	 *
 	 * @param index The position at which the first byte will be read.
 	 * @param dst The memory into which the memory will be copied.
 	 * @param offset The copying offset in the destination memory.
 	 * @param length The number of bytes to be copied.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large that the requested number of 
 	 *                                   bytes exceed the amount of memory between the index and the memory
 	 *                                   segment's end.
 	 */
-	public final void get(int index, byte[] dst, int offset, int length) {
-		// system arraycopy does the boundary checks anyways, no need to check extra
-		System.arraycopy(this.memory, index, dst, offset, length);
-	}
+	public abstract void get(int index, byte[] dst, int offset, int length);
 
 	/**
 	 * Bulk put method. Copies length memory starting at position offset from
 	 * the source memory into the memory segment starting at the specified
 	 * index.
-	 * 
+	 *
 	 * @param index The position in the memory segment array, where the data is put.
 	 * @param src The source array to copy the data from.
 	 * @param offset The offset in the source array where the copying is started.
 	 * @param length The number of bytes to copy.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or too large such that the array 
 	 *                                   portion to copy exceed the amount of memory between the index and the memory
 	 *                                   segment's end.
 	 */
-	public final void put(int index, byte[] src, int offset, int length) {
-		// system arraycopy does the boundary checks anyways, no need to check extra
-		System.arraycopy(src, offset, this.memory, index, length);
-	}
+	public abstract void put(int index, byte[] src, int offset, int length);
 
 	/**
 	 * Reads one byte at the given position and returns its boolean
 	 * representation.
-	 * 
+	 *
 	 * @param index The position from which the memory will be read.
 	 * @return The boolean value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 1.
 	 */
-	public final boolean getBoolean(int index) {
-		return this.memory[index] != 0;
-	}
+	public abstract boolean getBoolean(int index);
 
 	/**
 	 * Writes one byte containing the byte value into this buffer at the given
 	 * position.
-	 * 
+	 *
 	 * @param index The position at which the memory will be written.
 	 * @param value The char value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 1.
 	 */
-	public final void putBoolean(int index, boolean value) {
-		this.memory[index] = (byte) (value ? 1 : 0);
-	}
+	public abstract void putBoolean(int index, boolean value);
 
 	/**
-	 * Reads two memory at the given position, composing them into a char value
-	 * according to the current byte order.
-	 * 
+	 * Reads a char value from the given position, in the system's native byte order.
+	 *
 	 * @param index The position from which the memory will be read.
 	 * @return The char value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 2.
 	 */
+	@SuppressWarnings("restriction")
 	public final char getChar(int index) {
-		return (char) ( ((this.memory[index    ] & 0xff) << 8) | 
-						(this.memory[index + 1] & 0xff) );
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			return UNSAFE.getChar(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
 	}
 
 	/**
-	 * Writes two memory containing the given char value, in the current byte
-	 * order, into this buffer at the given position.
-	 * 
+	 * Reads an character value (16 bit, 2 bytes) from the given position, in little-endian byte order.
+	 * This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #getChar(int)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #getChar(int)} is the preferable choice.
+	 *
+	 * @param index The position from which the value will be read.
+	 * @return The character value at the given position.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final char getCharLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getChar(index);
+		} else {
+			return Character.reverseBytes(getChar(index));
+		}
+	}
+
+	/**
+	 * Reads an character value (16 bit, 2 bytes) from the given position, in big-endian byte order.
+	 * This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #getChar(int)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #getChar(int)} is the preferable choice.
+	 *
+	 * @param index The position from which the value will be read.
+	 * @return The character value at the given position.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final char getCharBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Character.reverseBytes(getChar(index));
+		} else {
+			return getChar(index);
+		}
+	}
+
+	/**
+	 * Writes a char value to teh given position, in the system's native byte order.
+	 *
 	 * @param index The position at which the memory will be written.
 	 * @param value The char value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 2.
 	 */
+	@SuppressWarnings("restriction")
 	public final void putChar(int index, char value) {
-		this.memory[index    ] = (byte) (value >> 8);
-		this.memory[index + 1] = (byte) value;
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			UNSAFE.putChar(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	/**
+	 * Writes the given character (16 bit, 2 bytes) to the given position in little-endian
+	 * byte order. This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #putChar(int, char)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #putChar(int, char)} is the preferable choice.
+	 *
+	 * @param index The position at which the value will be written.
+	 * @param value The short value to be written.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final void putCharLittleEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, value);
+		} else {
+			putChar(index, Character.reverseBytes(value));
+		}
+	}
+
+	/**
+	 * Writes the given character (16 bit, 2 bytes) to the given position in big-endian
+	 * byte order. This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #putChar(int, char)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #putChar(int, char)} is the preferable choice.
+	 *
+	 * @param index The position at which the value will be written.
+	 * @param value The short value to be written.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final void putCharBigEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, Character.reverseBytes(value));
+		} else {
+			putChar(index, value);
+		}
 	}
 
 	/**
 	 * Reads two memory at the given position, composing them into a short value
 	 * according to the current byte order.
-	 * 
+	 *
 	 * @param index The position from which the memory will be read.
 	 * @return The short value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 2.
 	 */
 	public final short getShort(int index) {
-		return (short) (
-				((this.memory[index    ] & 0xff) << 8) |
-				((this.memory[index + 1] & 0xff)) );
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			return UNSAFE.getShort(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	/**
+	 * Reads an short integer value (16 bit, 2 bytes) from the given position, in little-endian byte order.
+	 * This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #getShort(int)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #getShort(int)} is the preferable choice.
+	 *
+	 * @param index The position from which the value will be read.
+	 * @return The short value at the given position.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final short getShortLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getShort(index);
+		} else {
+			return Short.reverseBytes(getShort(index));
+		}
+	}
+
+	/**
+	 * Reads an short integer value (16 bit, 2 bytes) from the given position, in big-endian byte order.
+	 * This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #getShort(int)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #getShort(int)} is the preferable choice.
+	 *
+	 * @param index The position from which the value will be read.
+	 * @return The short value at the given position.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final short getShortBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Short.reverseBytes(getShort(index));
+		} else {
+			return getShort(index);
+		}
 	}
 
 	/**
 	 * Writes the given short value into this buffer at the given position, using
 	 * the native byte order of the system.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The short value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 2.
 	 */
 	public final void putShort(int index, short value) {
-		this.memory[index    ] = (byte) (value >> 8);
-		this.memory[index + 1] = (byte) value;
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			UNSAFE.putShort(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
 	}
-	
+
+	/**
+	 * Writes the given short integer value (16 bit, 2 bytes) to the given position in little-endian
+	 * byte order. This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #putShort(int, short)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #putShort(int, short)} is the preferable choice.
+	 *
+	 * @param index The position at which the value will be written.
+	 * @param value The short value to be written.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final void putShortLittleEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, value);
+		} else {
+			putShort(index, Short.reverseBytes(value));
+		}
+	}
+
+	/**
+	 * Writes the given short integer value (16 bit, 2 bytes) to the given position in big-endian
+	 * byte order. This method's speed depends on the system's native byte order, and it
+	 * is possibly slower than {@link #putShort(int, short)}. For most cases (such as 
+	 * transient storage in memory or serialization for I/O and network),
+	 * it suffices to know that the byte order in which the value is written is the same as the
+	 * one in which it is read, and {@link #putShort(int, short)} is the preferable choice.
+	 *
+	 * @param index The position at which the value will be written.
+	 * @param value The short value to be written.
+	 *
+	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment size minus 2.
+	 */
+	public final void putShortBigEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, Short.reverseBytes(value));
+		} else {
+			putShort(index, value);
+		}
+	}
+
 	/**
 	 * Reads an int value (32bit, 4 bytes) from the given position, in the system's native byte order.
 	 * This method offers the best speed for integer reading and should be used
@@ -353,37 +618,38 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The int value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
-	@SuppressWarnings("restriction")
 	public final int getInt(int index) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 4) {
-				return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 4) {
+			return UNSAFE.getInt(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
 		}
 	}
-	
+
 	/**
-	 * Reads an int value (32bit, 4 bytes) from the given position, in little endian byte order.
+	 * Reads an int value (32bit, 4 bytes) from the given position, in little-endian byte order.
 	 * This method's speed depends on the system's native byte order, and it
 	 * is possibly slower than {@link #getInt(int)}. For most cases (such as 
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getInt(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The int value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
@@ -394,18 +660,18 @@ public class MemorySegment {
 			return Integer.reverseBytes(getInt(index));
 		}
 	}
-	
+
 	/**
-	 * Reads an int value (32bit, 4 bytes) from the given position, in big endian byte order.
+	 * Reads an int value (32bit, 4 bytes) from the given position, in big-endian byte order.
 	 * This method's speed depends on the system's native byte order, and it
 	 * is possibly slower than {@link #getInt(int)}. For most cases (such as 
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getInt(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The int value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
@@ -424,26 +690,27 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The int value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
-	@SuppressWarnings("restriction")
 	public final void putInt(int index, int value) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 4) {
-				UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 4) {
+			UNSAFE.putInt(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
 		}
 	}
-	
+
 	/**
 	 * Writes the given int value (32bit, 4 bytes) to the given position in little endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -451,10 +718,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putInt(int, int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The int value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
@@ -465,7 +732,7 @@ public class MemorySegment {
 			putInt(index, Integer.reverseBytes(value));
 		}
 	}
-	
+
 	/**
 	 * Writes the given int value (32bit, 4 bytes) to the given position in big endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -473,10 +740,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putInt(int, int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The int value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
@@ -487,7 +754,7 @@ public class MemorySegment {
 			putInt(index, value);
 		}
 	}
-	
+
 	/**
 	 * Reads a long value (64bit, 8 bytes) from the given position, in the system's native byte order.
 	 * This method offers the best speed for long integer reading and should be used
@@ -495,26 +762,27 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The long value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
-	@SuppressWarnings("restriction")
 	public final long getLong(int index) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 8) {
-				return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 8) {
+			return UNSAFE.getLong(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
 		}
 	}
-	
+
 	/**
 	 * Reads a long integer value (64bit, 8 bytes) from the given position, in little endian byte order.
 	 * This method's speed depends on the system's native byte order, and it
@@ -522,10 +790,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getLong(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The long value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
@@ -536,7 +804,7 @@ public class MemorySegment {
 			return Long.reverseBytes(getLong(index));
 		}
 	}
-	
+
 	/**
 	 * Reads a long integer value (64bit, 8 bytes) from the given position, in big endian byte order.
 	 * This method's speed depends on the system's native byte order, and it
@@ -544,10 +812,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getLong(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The long value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
@@ -566,26 +834,27 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The long value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
-	@SuppressWarnings("restriction")
 	public final void putLong(int index, long value) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 8) {
-				UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 8) {
+			UNSAFE.putLong(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("segment has been freed");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
 		}
 	}
-	
+
 	/**
 	 * Writes the given long value (64bit, 8 bytes) to the given position in little endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -593,10 +862,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putLong(int, long)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The long value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
@@ -607,7 +876,7 @@ public class MemorySegment {
 			putLong(index, Long.reverseBytes(value));
 		}
 	}
-	
+
 	/**
 	 * Writes the given long value (64bit, 8 bytes) to the given position in big endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -615,10 +884,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putLong(int, long)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The long value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
@@ -629,7 +898,7 @@ public class MemorySegment {
 			putLong(index, value);
 		}
 	}
-	
+
 	/**
 	 * Reads a single-precision floating point value (32bit, 4 bytes) from the given position, in the system's
 	 * native byte order. This method offers the best speed for float reading and should be used
@@ -637,17 +906,17 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The float value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
 	public final float getFloat(int index) {
 		return Float.intBitsToFloat(getInt(index));
 	}
-	
+
 	/**
 	 * Reads a single-precision floating point value (32bit, 4 bytes) from the given position, in little endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -655,17 +924,17 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getFloat(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The long value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final float getFloatLittleEndian(int index) {
 		return Float.intBitsToFloat(getIntLittleEndian(index));
 	}
-	
+
 	/**
 	 * Reads a single-precision floating point value (32bit, 4 bytes) from the given position, in big endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -673,10 +942,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getFloat(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The long value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
@@ -691,17 +960,17 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The float value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 4.
 	 */
 	public final void putFloat(int index, float value) {
 		putInt(index, Float.floatToRawIntBits(value));
 	}
-	
+
 	/**
 	 * Writes the given single-precision float value (32bit, 4 bytes) to the given position in little endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -709,17 +978,17 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putFloat(int, float)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The long value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final void putFloatLittleEndian(int index, float value) {
 		putIntLittleEndian(index, Float.floatToRawIntBits(value));
 	}
-	
+
 	/**
 	 * Writes the given single-precision float value (32bit, 4 bytes) to the given position in big endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -727,17 +996,17 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putFloat(int, float)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The long value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final void putFloatBigEndian(int index, float value) {
 		putIntBigEndian(index, Float.floatToRawIntBits(value));
 	}
-	
+
 	/**
 	 * Reads a double-precision floating point value (64bit, 8 bytes) from the given position, in the system's
 	 * native byte order. This method offers the best speed for double reading and should be used
@@ -745,17 +1014,17 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The double value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final double getDouble(int index) {
 		return Double.longBitsToDouble(getLong(index));
 	}
-	
+
 	/**
 	 * Reads a double-precision floating point value (64bit, 8 bytes) from the given position, in little endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -763,17 +1032,17 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getDouble(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The long value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final double getDoubleLittleEndian(int index) {
 		return Double.longBitsToDouble(getLongLittleEndian(index));
 	}
-	
+
 	/**
 	 * Reads a double-precision floating point value (64bit, 8 bytes) from the given position, in big endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -781,10 +1050,10 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #getDouble(int)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position from which the value will be read.
 	 * @return The long value at the given position.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
@@ -799,17 +1068,17 @@ public class MemorySegment {
 	 * byte order in which the value is written is the same as the one in which it is read 
 	 * (such as transient storage in memory, or serialization for I/O and network), making this
 	 * method the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the memory will be written.
 	 * @param value The double value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final void putDouble(int index, double value) {
 		putLong(index, Double.doubleToRawLongBits(value));
 	}
-	
+
 	/**
 	 * Writes the given double-precision floating-point value (64bit, 8 bytes) to the given position in little endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -817,17 +1086,17 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putDouble(int, double)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The long value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final void putDoubleLittleEndian(int index, double value) {
 		putLongLittleEndian(index, Double.doubleToRawLongBits(value));
 	}
-	
+
 	/**
 	 * Writes the given double-precision floating-point value (64bit, 8 bytes) to the given position in big endian
 	 * byte order. This method's speed depends on the system's native byte order, and it
@@ -835,70 +1104,53 @@ public class MemorySegment {
 	 * transient storage in memory or serialization for I/O and network),
 	 * it suffices to know that the byte order in which the value is written is the same as the
 	 * one in which it is read, and {@link #putDouble(int, double)} is the preferable choice.
-	 * 
+	 *
 	 * @param index The position at which the value will be written.
 	 * @param value The long value to be written.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException Thrown, if the index is negative, or larger then the segment
 	 *                                   size minus 8.
 	 */
 	public final void putDoubleBigEndian(int index, double value) {
 		putLongBigEndian(index, Double.doubleToRawLongBits(value));
 	}
-	
+
 	// -------------------------------------------------------------------------
 	//                     Bulk Read and Write Methods
 	// -------------------------------------------------------------------------
-	
-	/**
-	 * Bulk get method. Copies length memory from the specified offset to the
-	 * provided <tt>DataOutput</tt>.
-	 * 
-	 * @param out The data output object to copy the data to.
-	 * @param offset The first byte to by copied.
-	 * @param length The number of bytes to copy.
-	 * 
-	 * @throws IOException Thrown, if the DataOutput encountered a problem upon writing.
-	 */
-	public final void get(DataOutput out, int offset, int length) throws IOException {
-		out.write(this.memory, offset, length);
-	}
+
+	public abstract void get(DataOutput out, int offset, int length) throws IOException;
 
 	/**
 	 * Bulk put method. Copies length memory from the given DataInput to the
 	 * memory starting at position offset.
-	 * 
+	 *
 	 * @param in The DataInput to get the data from.
 	 * @param offset The position in the memory segment to copy the chunk to.
 	 * @param length The number of bytes to get. 
-	 * 
+	 *
 	 * @throws IOException Thrown, if the DataInput encountered a problem upon reading,
 	 *                     such as an End-Of-File.
 	 */
-	public final void put(DataInput in, int offset, int length) throws IOException {
-		in.readFully(this.memory, offset, length);
-	}
-	
+	public abstract void put(DataInput in, int offset, int length) throws IOException;
+
 	/**
 	 * Bulk get method. Copies {@code numBytes} bytes from this memory segment, starting at position
 	 * {@code offset} to the target {@code ByteBuffer}. The bytes will be put into the target buffer
 	 * starting at the buffer's current position. If this method attempts to write more bytes than
 	 * the target byte buffer has remaining (with respect to {@link ByteBuffer#remaining()}),
 	 * this method will cause a {@link java.nio.BufferOverflowException}.
-	 * 
+	 *
 	 * @param offset The position where the bytes are started to be read from in this memory segment.
 	 * @param target The ByteBuffer to copy the bytes to.
 	 * @param numBytes The number of bytes to copy.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException If the offset is invalid, or this segment does not
 	 *           contain the given number of bytes (starting from offset), or the target byte buffer does
 	 *           not have enough space for the bytes.
 	 */
-	public final void get(int offset, ByteBuffer target, int numBytes) {
-		// ByteBuffer performs the boundy checks
-		target.put(this.memory, offset, numBytes);
-	}
-	
+	public abstract void get(int offset, ByteBuffer target, int numBytes);
+
 	/**
 	 * Bulk put method. Copies {@code numBytes} bytes from the given {@code ByteBuffer}, into
 	 * this memory segment. The bytes will be read from the target buffer
@@ -907,68 +1159,132 @@ public class MemorySegment {
 	 * If this method attempts to read more bytes than
 	 * the target byte buffer has remaining (with respect to {@link ByteBuffer#remaining()}),
 	 * this method will cause a {@link java.nio.BufferUnderflowException}.
-	 * 
+	 *
 	 * @param offset The position where the bytes are started to be written to in this memory segment.
 	 * @param source The ByteBuffer to copy the bytes from.
 	 * @param numBytes The number of bytes to copy.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException If the offset is invalid, or the source buffer does not
 	 *           contain the given number of bytes, or this segment does
 	 *           not have enough space for the bytes (counting from offset).
 	 */
-	public final void put(int offset, ByteBuffer source, int numBytes) {
-		// ByteBuffer performs the boundy checks
-		source.get(this.memory, offset, numBytes);
-	}
-	
+	public abstract void put(int offset, ByteBuffer source, int numBytes);
+
 	/**
 	 * Bulk copy method. Copies {@code numBytes} bytes from this memory segment, starting at position
 	 * {@code offset} to the target memory segment. The bytes will be put into the target segment
 	 * starting at position {@code targetOffset}.
-	 * 
+	 *
 	 * @param offset The position where the bytes are started to be read from in this memory segment.
 	 * @param target The memory segment to copy the bytes to.
 	 * @param targetOffset The position in the target memory segment to copy the chunk to.
 	 * @param numBytes The number of bytes to copy.
-	 * 
+	 *
 	 * @throws IndexOutOfBoundsException If either of the offsets is invalid, or the source segment does not
 	 *           contain the given number of bytes (starting from offset), or the target segment does
 	 *           not have enough space for the bytes (counting from targetOffset).
 	 */
 	public final void copyTo(int offset, MemorySegment target, int targetOffset, int numBytes) {
-		// system arraycopy does the boundary checks anyways, no need to check extra
-		System.arraycopy(this.memory, offset, target.memory, targetOffset, numBytes);
+		final byte[] thisHeapRef = this.heapMemory;
+		final byte[] otherHeapRef = target.heapMemory;
+		final long thisPointer = this.address + offset;
+		final long otherPointer = target.address + targetOffset;
+
+		if ( (numBytes | offset | targetOffset) >= 0 &&
+				thisPointer <= this.addressLimit - numBytes && otherPointer <= target.addressLimit - numBytes)
+		{
+			UNSAFE.copyMemory(thisHeapRef, thisPointer, otherHeapRef, otherPointer, numBytes);
+		}
+		else if (this.address > this.addressLimit) {
+			throw new IllegalStateException("this memory segment has been freed.");
+		}
+		else if (target.address > target.addressLimit) {
+			throw new IllegalStateException("target memory segment has been freed.");
+		}
+		else {
+			throw new IndexOutOfBoundsException(
+					String.format("offset=%d, targetOffset=%d, numBytes=%d, address=%d, targetAddress=%d",
+					offset, targetOffset, numBytes, this.address, target.address));
+		}
 	}
-	
+
 	// -------------------------------------------------------------------------
 	//                      Comparisons & Swapping
 	// -------------------------------------------------------------------------
-	
-	public static final int compare(MemorySegment seg1, MemorySegment seg2, int offset1, int offset2, int len) {
-		final byte[] b1 = seg1.memory;
-		final byte[] b2 = seg2.memory;
-		
-		int val = 0;
-		for (int pos = 0; pos < len && (val = (b1[offset1 + pos] & 0xff) - (b2[offset2 + pos] & 0xff)) == 0; pos++);
-		return val;
-	}
-	
-	public static final void swapBytes(MemorySegment seg1, MemorySegment seg2, byte[] tempBuffer, int offset1, int offset2, int len) {
-		// system arraycopy does the boundary checks anyways, no need to check extra
-		System.arraycopy(seg1.memory, offset1, tempBuffer, 0, len);
-		System.arraycopy(seg2.memory, offset2, seg1.memory, offset1, len);
-		System.arraycopy(tempBuffer, 0, seg2.memory, offset2, len);
+
+	/**
+	 * Compares two memory segment regions.
+	 *
+	 * @param seg2 Segment to compare this segment with
+	 * @param offset1 Offset of this segment to start comparing
+	 * @param offset2 Offset of seg2 to start comparing
+	 * @param len Length of the compared memory region
+	 *
+	 * @return 0 if equal, -1 if seg1 < seg2, 1 otherwise
+	 */
+	public final int compare(MemorySegment seg2, int offset1, int offset2, int len) {
+		while (len >= 8) {
+			long l1 = this.getLongBigEndian(offset1);
+			long l2 = seg2.getLongBigEndian(offset2);
+
+			if (l1 != l2) {
+				return (l1 < l2) ^ (l1 < 0) ^ (l2 < 0) ? -1 : 1;
+			}
+
+			offset1 += 8;
+			offset2 += 8;
+			len -= 8;
+		}
+		while (len > 0) {
+			int b1 = this.get(offset1) & 0xff;
+			int b2 = seg2.get(offset2) & 0xff;
+			int cmp = b1 - b2;
+			if (cmp != 0) {
+				return cmp;
+			}
+			offset1++;
+			offset2++;
+			len--;
+		}
+		return 0;
 	}
+
+	/**
+	 * Swaps bytes between two memory segments, using the given auxiliary buffer.
+	 *
+	 * @param tempBuffer The auxiliary buffer in which to put data during triangle swap.
+	 * @param seg2 Segment to swap bytes with
+	 * @param offset1 Offset of this segment to start swapping
+	 * @param offset2 Offset of seg2 to start swapping
+	 * @param len Length of the swapped memory region
+	 */
+	public final void swapBytes(byte[] tempBuffer, MemorySegment seg2, int offset1, int offset2, int len) {
+		if ( (offset1 | offset2 | len | (tempBuffer.length - len) ) >= 0) {
+			final long thisPos = this.address + offset1;
+			final long otherPos = seg2.address + offset2;
+			
+			if (thisPos <= this.addressLimit - len && otherPos <= seg2.addressLimit - len) {
+				// this -> temp buffer
+				UNSAFE.copyMemory(this.heapMemory, thisPos, tempBuffer, BYTE_ARRAY_BASE_OFFSET, len);
 	
-	// --------------------------------------------------------------------------------------------
-	//                     Utilities for native memory accesses and checks
-	// --------------------------------------------------------------------------------------------
-	
-	@SuppressWarnings("restriction")
-	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
-	
-	@SuppressWarnings("restriction")
-	private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+				// other -> this
+				UNSAFE.copyMemory(seg2.heapMemory, otherPos, this.heapMemory, thisPos, len);
 	
-	private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
+				// temp buffer -> other
+				UNSAFE.copyMemory(tempBuffer, BYTE_ARRAY_BASE_OFFSET, seg2.heapMemory, otherPos, len);
+				return;
+			}
+			else if (this.address > this.addressLimit) {
+				throw new IllegalStateException("this memory segment has been freed.");
+			}
+			else if (seg2.address > seg2.addressLimit) {
+				throw new IllegalStateException("other memory segment has been freed.");
+			}
+		}
+		
+		// index is in fact invalid
+		throw new IndexOutOfBoundsException(
+					String.format("offset1=%d, offset2=%d, len=%d, bufferSize=%d, address1=%d, address2=%d",
+							offset1, offset2, len, tempBuffer.length, this.address, seg2.address));
+	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
new file mode 100644
index 0000000..0e4e469
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemorySegmentFactory.java
@@ -0,0 +1,211 @@
+/*
+ * 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 java.nio.ByteBuffer;
+
+/**
+ * A factory for memory segments. The purpose of this factory is to make sure that all memory segments
+ * for heap data are of the same type. That way, the runtime does not mix the various specializations
+ * of the {@link org.apache.flink.core.memory.MemorySegment}. Not mixing them has shown to be beneficial
+ * to method specialization by the JIT and to overall performance.
+ * <p>
+ * Note that this factory auto-initialized to use {@link org.apache.flink.core.memory.HeapMemorySegment},
+ * if a request to create a segment comes before the initialization.
+ */
+public class MemorySegmentFactory {
+
+	/** The factory to use */
+	private static volatile Factory factory;
+	
+	/**
+	 * Creates a new memory segment that targets the given heap memory region.
+	 * This method should be used to turn short lived byte arrays into memory segments.
+	 *
+	 * @param buffer The heap memory region.
+	 * @return A new memory segment that targets the given heap memory region.
+	 */
+	public static MemorySegment wrap(byte[] buffer) {
+		ensureInitialized();
+		return factory.wrap(buffer);
+	}
+
+	/**
+	 * Allocates some unpooled memory and creates a new memory segment that represents
+	 * that memory.
+	 * <p>
+	 * This method is similar to {@link #allocateUnpooledSegment(int, Object)}, but the
+	 * memory segment will have null as the owner.
+	 *
+	 * @param size The size of the memory segment to allocate.
+	 * @return A new memory segment, backed by unpooled heap memory.
+	 */
+	public static MemorySegment allocateUnpooledSegment(int size) {
+		return allocateUnpooledSegment(size, null);
+	}
+
+	/**
+	 * Allocates some unpooled memory and creates a new memory segment that represents
+	 * that memory.
+	 * <p>
+	 * This method is similar to {@link #allocateUnpooledSegment(int)}, but additionally sets
+	 * the owner of the memory segment.
+	 * 
+	 * @param size The size of the memory segment to allocate.
+	 * @param owner The owner to associate with the memory segment.
+	 * @return A new memory segment, backed by unpooled heap memory.
+	 */
+	public static MemorySegment allocateUnpooledSegment(int size, Object owner) {
+		ensureInitialized();
+		return factory.allocateUnpooledSegment(size, owner);
+	}
+
+	/**
+	 * Creates a memory segment that wraps the given byte array.
+	 * <p>
+	 * This method is intended to be used for components which pool memory and create
+	 * memory segments around long-lived memory regions.
+	 *
+	 * 
+	 * @param memory The heap memory to be represented by the memory segment.
+	 * @param owner The owner to associate with the memory segment.
+	 * @return A new memory segment representing the given heap memory.
+	 */
+	public static MemorySegment wrapPooledHeapMemory(byte[] memory, Object owner) {
+		ensureInitialized();
+		return factory.wrapPooledHeapMemory(memory, owner);
+	}
+
+	/**
+	 * Creates a memory segment that wraps the off-heap memory backing the given ByteBuffer.
+	 * Note that the ByteBuffer needs to be a <i>direct ByteBuffer</i>. 
+	 * <p>
+	 * This method is intended to be used for components which pool memory and create
+	 * memory segments around long-lived memory regions.
+	 *
+	 * @param memory The byte buffer with the off-heap memory to be represented by the memory segment.
+	 * @param owner The owner to associate with the memory segment.
+	 * @return A new memory segment representing the given off-heap memory.
+	 */
+	public static MemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner) {
+		ensureInitialized();
+		return factory.wrapPooledOffHeapMemory(memory, owner);
+	}
+	
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * Initializes this factory with the given concrete factory.
+	 * 
+	 * @param f The concrete factory to use.
+	 * @throws java.lang.IllegalStateException Thrown, if this factory has been initialized before.
+	 */
+	public static void initializeFactory(Factory f) {
+		if (f == null) {
+			throw new NullPointerException();
+		}
+	
+		synchronized (MemorySegmentFactory.class) {
+			if (factory == null) {
+				factory = f;
+			}
+			else {
+				throw new IllegalStateException("Factory has already been initialized");
+			}
+		}
+	}
+
+	/**
+	 * Checks whether this memory segment factory has been initialized (with a type to produce).
+	 * 
+	 * @return True, if the factory has been initialized, false otherwise.
+	 */
+	public static boolean isInitialized() {
+		return factory != null;
+	}
+
+	/**
+	 * Gets the factory. May return null, if the factory has not been initialized.
+	 * 
+	 * @return The factory, or null, if the factory has not been initialized.
+	 */
+	public static Factory getFactory() {
+		return factory;
+	}
+	
+	private static void ensureInitialized() {
+		if (factory == null) {
+			factory = HeapMemorySegment.FACTORY;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Internal factory
+	// ------------------------------------------------------------------------
+	
+	/**
+	 * A concrete factory for memory segments.
+	 */
+	public static interface Factory {
+
+		/**
+		 * Creates a new memory segment that targets the given heap memory region.
+		 *
+		 * @param memory The heap memory region.
+		 * @return A new memory segment that targets the given heap memory region.
+		 */
+		MemorySegment wrap(byte[] memory);
+
+		/**
+		 * Allocates some unpooled memory and creates a new memory segment that represents
+		 * that memory.
+		 *
+		 * @param size The size of the memory segment to allocate.
+		 * @param owner The owner to associate with the memory segment.
+		 * @return A new memory segment, backed by unpooled heap memory.
+		 */
+		MemorySegment allocateUnpooledSegment(int size, Object owner);
+
+		/**
+		 * Creates a memory segment that wraps the given byte array.
+		 * <p>
+		 * This method is intended to be used for components which pool memory and create
+		 * memory segments around long-lived memory regions.
+		 *
+		 *
+		 * @param memory The heap memory to be represented by the memory segment.
+		 * @param owner The owner to associate with the memory segment.
+		 * @return A new memory segment representing the given heap memory.
+		 */
+		MemorySegment wrapPooledHeapMemory(byte[] memory, Object owner);
+
+		/**
+		 * Creates a memory segment that wraps the off-heap memory backing the given ByteBuffer.
+		 * Note that the ByteBuffer needs to be a <i>direct ByteBuffer</i>. 
+		 * <p>
+		 * This method is intended to be used for components which pool memory and create
+		 * memory segments around long-lived memory regions.
+		 *
+		 * @param memory The byte buffer with the off-heap memory to be represented by the memory segment.
+		 * @param owner The owner to associate with the memory segment.
+		 * @return A new memory segment representing the given off-heap memory.
+		 */
+		MemorySegment wrapPooledOffHeapMemory(ByteBuffer memory, Object owner);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java
new file mode 100644
index 0000000..5b2b5d3
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryType.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+/**
+ * The class of memory, such as heap or off-heap.
+ */
+public enum MemoryType {
+
+	/**
+	 * Denotes memory that is part of the Java heap.
+	 */
+	HEAP,
+
+	/**
+	 * Denotes memory that is outside the Java heap (but still part of tha Java process).
+	 */
+	OFF_HEAP
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
index c1f626f..20b37c9 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/MemoryUtils.java
@@ -22,22 +22,17 @@ import java.lang.reflect.Field;
 import java.nio.ByteOrder;
 
 /**
- * Utility class for native (unsafe) memory accesses.
+ * Utility class for memory operations.
  */
 public class MemoryUtils {
 	
-	/**
-	 * The "unsafe", which can be used to perform native memory accesses.
-	 */
+	/** The "unsafe", which can be used to perform native memory accesses. */
 	@SuppressWarnings("restriction")
 	public static final sun.misc.Unsafe UNSAFE = getUnsafe();
 	
-	/**
-	 * The native byte order of the platform on which the system currently runs.
-	 */
-	public static final ByteOrder NATIVE_BYTE_ORDER = getByteOrder();
-	
-	
+	/** The native byte order of the platform on which the system currently runs. */
+	public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+
 	@SuppressWarnings("restriction")
 	private static sun.misc.Unsafe getUnsafe() {
 		try {
@@ -45,21 +40,18 @@ public class MemoryUtils {
 			unsafeField.setAccessible(true);
 			return (sun.misc.Unsafe) unsafeField.get(null);
 		} catch (SecurityException e) {
-			throw new RuntimeException("Could not access the unsafe handle.", e);
+			throw new RuntimeException("Could not access the sun.misc.Unsafe handle, permission denied by security manager.", e);
 		} catch (NoSuchFieldException e) {
-			throw new RuntimeException("The static unsafe handle field was not be found.");
+			throw new RuntimeException("The static handle field in sun.misc.Unsafe was not found.");
 		} catch (IllegalArgumentException e) {
-			throw new RuntimeException("Bug: Illegal argument reflection access for static field.");
+			throw new RuntimeException("Bug: Illegal argument reflection access for static field.", e);
 		} catch (IllegalAccessException e) {
-			throw new RuntimeException("Access to the unsafe handle is forbidden by the runtime.", e);
+			throw new RuntimeException("Access to sun.misc.Unsafe is forbidden by the runtime.", e);
+		} catch (Throwable t) {
+			throw new RuntimeException("Unclassified error while trying to access the sun.misc.Unsafe handle.", t);
 		}
 	}
-	
-	@SuppressWarnings("restriction")
-	private static ByteOrder getByteOrder() {
-		return ByteOrder.nativeOrder();
-	}
-	
-	
+
+	/** Should not be instantiated */
 	private MemoryUtils() {}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
index 782c402..a8ace92 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/ComparatorTestBase.java
@@ -29,7 +29,9 @@ import static org.junit.Assert.*;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.util.TestLogger;
+
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -238,7 +240,7 @@ public abstract class ComparatorTestBase<T> extends TestLogger {
 	
 	// Help Function for setting up a memory segment and normalize the keys of the data array in it
 	public MemorySegment setupNormalizedKeysMemSegment(T[] data, int normKeyLen, TypeComparator<T> comparator) {
-		MemorySegment memSeg = new MemorySegment(new byte[2048]);
+		MemorySegment memSeg = MemorySegmentFactory.allocateUnpooledSegment(2048);
 
 		// Setup normalized Keys in the memory segment
 		int offset = 0;
@@ -294,7 +296,7 @@ public abstract class ComparatorTestBase<T> extends TestLogger {
 			MemorySegment memSeg2 = setupNormalizedKeysMemSegment(data, normKeyLen, comparator);
 
 			for (int i = 0; i < data.length; i++) {
-				assertTrue(MemorySegment.compare(memSeg1, memSeg2, i * normKeyLen, i * normKeyLen, normKeyLen) == 0);
+				assertTrue(memSeg1.compare(memSeg2, i * normKeyLen, i * normKeyLen, normKeyLen) == 0);
 			}
 		} catch (Exception e) {
 			System.err.println(e.getMessage());
@@ -343,14 +345,14 @@ public abstract class ComparatorTestBase<T> extends TestLogger {
 				for (int h = l + 1; h < data.length; h++) {
 					int cmp;
 					if (greater) {
-						cmp = MemorySegment.compare(memSegLow, memSegHigh, l * normKeyLen, h * normKeyLen, normKeyLen);
+						cmp = memSegLow.compare(memSegHigh, l * normKeyLen, h * normKeyLen, normKeyLen);
 						if (fullyDetermines) {
 							assertTrue(cmp < 0);
 						} else {
 							assertTrue(cmp <= 0);
 						}
 					} else {
-						cmp = MemorySegment.compare(memSegHigh, memSegLow, h * normKeyLen, l * normKeyLen, normKeyLen);
+						cmp = memSegHigh.compare(memSegLow, h * normKeyLen, l * normKeyLen, normKeyLen);
 						if (fullyDetermines) {
 							assertTrue(cmp > 0);
 						} else {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java
new file mode 100644
index 0000000..724a366
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/CrossSegmentTypeTest.java
@@ -0,0 +1,356 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Random;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class CrossSegmentTypeTest {
+
+	private final int pageSize = 32*1024;
+	
+	// ------------------------------------------------------------------------
+	
+	@Test
+	public void testCompareBytesMixedSegments() {
+		try {
+			MemorySegment[] segs1 = {
+					new HeapMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+			};
+
+			MemorySegment[] segs2 = {
+					new HeapMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+			};
+		
+			Random rnd = new Random();
+			
+			for (MemorySegment seg1 : segs1) {
+				for (MemorySegment seg2 : segs2) {
+					testCompare(seg1, seg2, rnd);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private void testCompare(MemorySegment seg1, MemorySegment seg2, Random random) {
+		assertEquals(pageSize, seg1.size());
+		assertEquals(pageSize, seg2.size());
+		
+		final byte[] bytes1 = new byte[pageSize];
+		final byte[] bytes2 = new byte[pageSize];
+
+		final int stride = pageSize / 255;
+		final int shift = 16666;
+
+		for (int i = 0; i < pageSize; i++) {
+			byte val = (byte) ((i / stride) & 0xff);
+			bytes1[i] = val;
+
+			if (i + shift < bytes2.length) {
+				bytes2[i + shift] = val;
+			}
+		}
+		
+		seg1.put(0, bytes1);
+		seg2.put(0, bytes2);
+
+		for (int i = 0; i < 1000; i++) {
+			int pos1 = random.nextInt(bytes1.length);
+			int pos2 = random.nextInt(bytes2.length);
+
+			int len = Math.min(Math.min(bytes1.length - pos1, bytes2.length - pos2),
+					random.nextInt(pageSize / 50 ));
+
+			int cmp = seg1.compare(seg2, pos1, pos2, len);
+
+			if (pos1 < pos2 - shift) {
+				assertTrue(cmp <= 0);
+			}
+			else {
+				assertTrue(cmp >= 0);
+			}
+		}
+	}
+
+
+	@Test
+	public void testSwapBytesMixedSegments() {
+		try {
+			final int HALF_SIZE = pageSize / 2;
+			
+			MemorySegment[] segs1 = {
+					new HeapMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+			};
+
+			MemorySegment[] segs2 = {
+					new HeapMemorySegment(new byte[HALF_SIZE]),
+					new HybridMemorySegment(new byte[HALF_SIZE]),
+					new HybridMemorySegment(ByteBuffer.allocateDirect(HALF_SIZE))
+			};
+
+			Random rnd = new Random();
+
+			for (MemorySegment seg1 : segs1) {
+				for (MemorySegment seg2 : segs2) {
+					testSwap(seg1, seg2, rnd, HALF_SIZE);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private void testSwap(MemorySegment seg1, MemorySegment seg2, Random random, int smallerSize) {
+		assertEquals(pageSize, seg1.size());
+		assertEquals(smallerSize, seg2.size());
+
+		final byte[] bytes1 = new byte[pageSize];
+		final byte[] bytes2 = new byte[smallerSize];
+
+		Arrays.fill(bytes2, (byte) 1);
+		
+		seg1.put(0, bytes1);
+		seg2.put(0, bytes2);
+
+		// wap the second half of the first segment with the second segment
+
+		int pos = 0;
+		while (pos < smallerSize) {
+			int len = random.nextInt(pageSize / 40);
+			len = Math.min(len, smallerSize - pos);
+			seg1.swapBytes(new byte[len], seg2, pos + smallerSize, pos, len);
+			pos += len;
+		}
+
+		// the second segment should now be all zeros, the first segment should have one in its second half
+
+		for (int i = 0; i < smallerSize; i++) {
+			assertEquals((byte) 0, seg1.get(i));
+			assertEquals((byte) 0, seg2.get(i));
+			assertEquals((byte) 1, seg1.get(i + smallerSize));
+		}
+	}
+
+	@Test
+	public void testCopyMixedSegments() {
+		try {
+			MemorySegment[] segs1 = {
+					new HeapMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+			};
+
+			MemorySegment[] segs2 = {
+					new HeapMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(new byte[pageSize]),
+					new HybridMemorySegment(ByteBuffer.allocateDirect(pageSize))
+			};
+
+			Random rnd = new Random();
+
+			for (MemorySegment seg1 : segs1) {
+				for (MemorySegment seg2 : segs2) {
+					testCopy(seg1, seg2, rnd);
+				}
+			}
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+
+	private void testCopy(MemorySegment seg1, MemorySegment seg2, Random random) {
+		assertEquals(pageSize, seg1.size());
+		assertEquals(pageSize, seg2.size());
+
+		byte[] expected = new byte[pageSize];
+		byte[] actual = new byte[pageSize];
+		
+		// zero out the memory
+		seg1.put(0, expected);
+		seg2.put(0, expected);
+		
+		for (int i = 0; i < 40; i++) {
+			int numBytes = random.nextInt(pageSize / 20);
+			byte[] bytes = new byte[numBytes];
+			random.nextBytes(bytes);
+			
+			int thisPos = random.nextInt(pageSize - numBytes);
+			int otherPos = random.nextInt(pageSize - numBytes);
+			
+			// track what we expect
+			System.arraycopy(bytes, 0, expected, otherPos, numBytes);
+			
+			seg1.put(thisPos, bytes);
+			seg1.copyTo(thisPos, seg2, otherPos, numBytes);
+		}
+		
+		seg2.get(0, actual);
+		assertArrayEquals(expected, actual);
+		
+		// test out of bound conditions
+
+		final int[] validOffsets = { 0, 1, pageSize / 10 * 9 };
+		final int[] invalidOffsets = { -1, pageSize + 1, -pageSize, Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+		final int[] validLengths = { 0, 1, pageSize / 10, pageSize };
+		final int[] invalidLengths = { -1, -pageSize, pageSize + 1, Integer.MAX_VALUE, Integer.MIN_VALUE };
+
+		for (int off1 : validOffsets) {
+			for (int off2 : validOffsets) {
+				for (int len : invalidLengths) {
+					try {
+						seg1.copyTo(off1, seg2, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg1.copyTo(off2, seg2, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+					
+					try {
+						seg2.copyTo(off1, seg1, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg2.copyTo(off2, seg1, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+				}
+			}
+		}
+
+		for (int off1 : validOffsets) {
+			for (int off2 : invalidOffsets) {
+				for (int len : validLengths) {
+					try {
+						seg1.copyTo(off1, seg2, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg1.copyTo(off2, seg2, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg2.copyTo(off1, seg1, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg2.copyTo(off2, seg1, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+				}
+			}
+		}
+
+		for (int off1 : invalidOffsets) {
+			for (int off2 : validOffsets) {
+				for (int len : validLengths) {
+					try {
+						seg1.copyTo(off1, seg2, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg1.copyTo(off2, seg2, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg2.copyTo(off1, seg1, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg2.copyTo(off2, seg1, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+				}
+			}
+		}
+
+		for (int off1 : invalidOffsets) {
+			for (int off2 : invalidOffsets) {
+				for (int len : validLengths) {
+					try {
+						seg1.copyTo(off1, seg2, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg1.copyTo(off2, seg2, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg2.copyTo(off1, seg1, off2, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+
+					try {
+						seg2.copyTo(off2, seg1, off1, len);
+						fail("should fail with an IndexOutOfBoundsException");
+					}
+					catch (IndexOutOfBoundsException ignored) {}
+				}
+			}
+		}
+	}
+}


[03/15] flink git commit: [FLINK-2635] [streaming] Make input processors independent of batch reader interface.

Posted by se...@apache.org.
[FLINK-2635] [streaming] Make input processors independent of batch reader interface.


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

Branch: refs/heads/master
Commit: b18e410bc336eb46497aaa75633c16ebe6139554
Parents: c09d14a
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 8 16:19:24 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200

----------------------------------------------------------------------
 .../runtime/io/StreamInputProcessor.java        | 22 +++++++++--------
 .../runtime/io/StreamTwoInputProcessor.java     | 25 +++++++-------------
 .../flink/streaming/api/PartitionerTest.java    |  7 +++---
 3 files changed, 25 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/b18e410b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index cc91d63..8ce8a01 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.reader.AbstractReader;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
 import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
@@ -52,19 +52,21 @@ import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
  * 
  * @param <IN> The type of the record that can be read with this record reader.
  */
-public class StreamInputProcessor<IN> extends AbstractReader implements StreamingReader {
+public class StreamInputProcessor<IN> {
 	
 	private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
 
 	private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
 
+	private final CheckpointBarrierHandler barrierHandler;
+
 	// We need to keep track of the channel from which a buffer came, so that we can
 	// appropriately map the watermarks to input channels
 	private int currentChannel = -1;
 
 	private boolean isFinished;
 
-	private final CheckpointBarrierHandler barrierHandler;
+	
 
 	private final long[] watermarks;
 	private long lastEmittedWatermark;
@@ -77,8 +79,8 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 								CheckpointingMode checkpointMode,
 								IOManager ioManager,
 								boolean enableWatermarkMultiplexing) throws IOException {
-		
-		super(InputGateUtil.createInputGate(inputGates));
+
+		InputGate inputGate = InputGateUtil.createInputGate(inputGates);
 
 		if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
 			this.barrierHandler = new BarrierBuffer(inputGate, ioManager);
@@ -173,7 +175,9 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 				else {
 					// Event received
 					final AbstractEvent event = bufferOrEvent.getEvent();
-					handleEvent(event);
+					if (event.getClass() != EndOfPartitionEvent.class) {
+						throw new IOException("Unexpected event: " + event);
+					}
 				}
 			}
 			else {
@@ -185,15 +189,13 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 			}
 		}
 	}
-
-	@Override
+	
 	public void setReporter(AccumulatorRegistry.Reporter reporter) {
 		for (RecordDeserializer<?> deserializer : recordDeserializers) {
 			deserializer.setReporter(reporter);
 		}
 	}
-
-	@Override
+	
 	public void cleanup() throws IOException {
 		// clear the buffers first. this part should not ever fail
 		for (RecordDeserializer<?> deserializer : recordDeserializers) {

http://git-wip-us.apache.org/repos/asf/flink/blob/b18e410b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
index 7dffa71..6322cc8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
@@ -22,8 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.io.network.api.reader.AbstractReader;
-import org.apache.flink.runtime.io.network.api.reader.ReaderBase;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer;
 import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
 import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer;
@@ -41,8 +40,6 @@ import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordS
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -58,10 +55,7 @@ import java.util.Collection;
  * @param <IN1> The type of the records that arrive on the first input
  * @param <IN2> The type of the records that arrive on the second input
  */
-public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements ReaderBase, StreamingReader {
-
-	@SuppressWarnings("unused")
-	private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputProcessor.class);
+public class StreamTwoInputProcessor<IN1, IN2> {
 
 	private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
 
@@ -97,7 +91,7 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 			IOManager ioManager,
 			boolean enableWatermarkMultiplexing) throws IOException {
 		
-		super(InputGateUtil.createInputGate(inputGates1, inputGates2));
+		final InputGate inputGate = InputGateUtil.createInputGate(inputGates1, inputGates2);
 
 		if (checkpointMode == CheckpointingMode.EXACTLY_ONCE) {
 			this.barrierHandler = new BarrierBuffer(inputGate, ioManager);
@@ -157,8 +151,7 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 		Arrays.fill(watermarks2, Long.MIN_VALUE);
 		lastEmittedWatermark2 = Long.MIN_VALUE;
 	}
-
-	@SuppressWarnings("unchecked")
+	
 	public boolean processInput(TwoInputStreamOperator<IN1, IN2, ?> streamOperator) throws Exception {
 		if (isFinished) {
 			return false;
@@ -216,7 +209,9 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 				} else {
 					// Event received
 					final AbstractEvent event = bufferOrEvent.getEvent();
-					handleEvent(event);
+					if (event.getClass() != EndOfPartitionEvent.class) {
+						throw new IOException("Unexpected event: " + event);
+					}
 				}
 			}
 			else {
@@ -259,15 +254,13 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 			}
 		}
 	}
-
-	@Override
+	
 	public void setReporter(AccumulatorRegistry.Reporter reporter) {
 		for (RecordDeserializer<?> deserializer : recordDeserializers) {
 			deserializer.setReporter(reporter);
 		}
 	}
-
-	@Override
+	
 	public void cleanup() throws IOException {
 		// clear the buffers first. this part should not ever fail
 		for (RecordDeserializer<?> deserializer : recordDeserializers) {

http://git-wip-us.apache.org/repos/asf/flink/blob/b18e410b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
index 987a8fb..a6c6936 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/PartitionerTest.java
@@ -32,13 +32,12 @@ import org.apache.flink.api.common.functions.RuntimeContext;
 import org.apache.flink.api.java.tuple.Tuple1;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.util.NoOpIntMap;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestListResultSink;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
+
 import org.junit.Test;
 
 /**
@@ -147,7 +146,9 @@ public class PartitionerTest extends StreamingMultipleProgramsTestBase {
 
 		try {
 			env.execute();
-		} catch (Exception e) {
+		}
+		catch (Exception e) {
+			e.printStackTrace();
 			fail(e.getMessage());
 		}
 


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java
new file mode 100644
index 0000000..57817b9
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java
@@ -0,0 +1,887 @@
+/*
+ * 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.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public final class PureHybridMemorySegment {
+
+	/** Constant that flags the byte order. Because this is a boolean constant,
+	 * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+	private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+	
+	/** The direct byte buffer that allocated the off-heap memory. This memory segment holds a reference
+	 * to that buffer, so as long as this memory segment lives, the memory will not be released. */
+	private final ByteBuffer offHeapMemory;
+	
+	/** The heap byte array object relative to which we access the memory. Is non-null if the
+	 *  memory is on the heap, is null, if the memory if off the heap. If we have this buffer, we
+	 *  must never void this reference, or the memory segment will point to undefined addresses 
+	 *  outside the heap and may in out-of-order execution cases cause segmentation faults. */
+	private final byte[] heapMemory;
+
+	/** The address to the data, relative to the heap memory byte array. If the heap memory byte array
+	 * is null, this becomes an absolute memory address outside the heap. */
+	private long address;
+
+	/** The address one byte after the last addressable byte.
+	 *  This is address + size while the segment is not disposed */
+	private final long addressLimit;
+
+	/** The size in bytes of the memory segment */
+	private final int size;
+
+	// -------------------------------------------------------------------------
+	//                             Constructors
+	// -------------------------------------------------------------------------
+
+	/**
+	 * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+	 * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+	 * otherwise this method with throw an IllegalArgumentException.
+	 *
+	 * @param buffer The byte buffer whose memory is represented by this memory segment.
+	 * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+	 */
+	public PureHybridMemorySegment(ByteBuffer buffer) {
+		if (buffer == null || !buffer.isDirect()) {
+			throw new IllegalArgumentException("Can't initialize from non-direct ByteBuffer.");
+		}
+
+		this.offHeapMemory = buffer;
+		this.heapMemory = null;
+		this.size = buffer.capacity();
+		this.address = getAddress(buffer);
+		this.addressLimit = this.address + size;
+
+		if (address >= Long.MAX_VALUE - Integer.MAX_VALUE) {
+			throw new RuntimeException("Segment initialized with too large address: " + address
+					+ " ; Max allowed address is " + (Long.MAX_VALUE - Integer.MAX_VALUE - 1));
+		}
+	}
+
+	/**
+	 * Creates a new memory segment that represents the memory of the byte array.
+	 *
+	 * @param buffer The byte array whose memory is represented by this memory segment.
+	 */
+	public PureHybridMemorySegment(byte[] buffer) {
+		if (buffer == null) {
+			throw new NullPointerException("buffer");
+		}
+		
+		this.offHeapMemory = null;
+		this.heapMemory = buffer;
+		this.address = BYTE_ARRAY_BASE_OFFSET;
+		this.addressLimit = BYTE_ARRAY_BASE_OFFSET + buffer.length;
+		this.size = buffer.length;
+	}
+	
+	// -------------------------------------------------------------------------
+	//                      Memory Segment Specifics
+	// -------------------------------------------------------------------------
+
+	/**
+	 * Gets the size of the memory segment, in bytes.
+	 * @return The size of the memory segment.
+	 */
+	public final int size() {
+		return size;
+	}
+
+	/**
+	 * Checks whether the memory segment was freed.
+	 * @return True, if the memory segment has been freed, false otherwise.
+	 */
+	public final boolean isFreed() {
+		return this.address > this.addressLimit;
+	}
+
+	/**
+	 * Frees this memory segment. After this operation has been called, no further operations are
+	 * possible on the memory segment and will fail. The actual memory (heap or off-heap) will only
+	 * be released after this memory segment object has become garbage collected. 
+	 */
+	public final void free() {
+		// this ensures we can place no more data and trigger
+		// the checks for the freed segment
+		address = addressLimit + 1;
+	}
+	
+	/**
+	 * Checks whether this memory segment is backed by off-heap memory.
+	 * @return True, if the memory segment is backed by off-heap memory, false if it is backed
+	 *         by heap memory.
+	 */
+	public final boolean isOffHeap() {
+		return heapMemory == null;
+	}
+
+	public byte[] getArray() {
+		if (heapMemory != null) {
+			return heapMemory;
+		} else {
+			throw new IllegalStateException("Memory segment does not represent heap memory");
+		}
+	}
+	
+	/**
+	 * Gets the buffer that owns the memory of this memory segment.
+	 *
+	 * @return The byte buffer that owns the memory of this memory segment.
+	 */
+	public ByteBuffer getOffHeapBuffer() {
+		if (offHeapMemory != null) {
+			return offHeapMemory;
+		} else {
+			throw new IllegalStateException("Memory segment does not represent off heap memory");
+		}
+	}
+	
+	public ByteBuffer wrap(int offset, int length) {
+		if (offset < 0 || offset > this.size || offset > this.size - length) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		if (heapMemory != null) {
+			return ByteBuffer.wrap(heapMemory, offset, length);
+		}
+		else {
+			ByteBuffer wrapper = offHeapMemory.duplicate();
+			wrapper.limit(offset + length);
+			wrapper.position(offset);
+			return wrapper;
+		}
+	}
+
+	/**
+	 * Gets this memory segment as a pure heap memory segment.
+	 * 
+	 * @return A heap memory segment variant of this memory segment.
+	 * @throws UnsupportedOperationException Thrown, if this memory segment is not
+	 *                                       backed by heap memory.
+	 */
+	public final PureHeapMemorySegment asHeapSegment() {
+		if (heapMemory != null) {
+			return new PureHeapMemorySegment(heapMemory);
+		} else {
+			throw new UnsupportedOperationException("Memory segment is not backed by heap memory");
+		}
+	}
+
+	/**
+	 * Gets this memory segment as a pure off-heap memory segment.
+	 *
+	 * @return An off-heap memory segment variant of this memory segment.
+	 * @throws UnsupportedOperationException Thrown, if this memory segment is not
+	 *                                       backed by off-heap memory.
+	 */
+	public final PureOffHeapMemorySegment asOffHeapSegment() {
+		if (offHeapMemory != null) {
+			return new PureOffHeapMemorySegment(offHeapMemory);
+		} else {
+			throw new UnsupportedOperationException("Memory segment is not backed by off-heap memory");
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//                    Random Access get() and put() methods
+	// ------------------------------------------------------------------------
+	
+	@SuppressWarnings("restriction")
+	public final byte get(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos < addressLimit) {
+			return UNSAFE.getByte(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void put(int index, byte b) {
+		final long pos = address + index;
+		if (index >= 0 && pos < addressLimit) {
+			UNSAFE.putByte(heapMemory, pos, b);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+	
+	public final void get(int index, byte[] dst) {
+		get(index, dst, 0, dst.length);
+	}
+	
+	public final void put(int index, byte[] src) {
+		put(index, src, 0, src.length);
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void get(int index, byte[] dst, int offset, int length) {
+		// check the byte array offset and length
+		if ((offset | length | (offset + length) | (dst.length - (offset + length))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		long pos = address + index;
+
+		if (index >= 0 && pos <= addressLimit - length) {
+			long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+
+			// the copy must proceed in batches not too large, because the JVM may
+			// poll for points that are safe for GC (moving the array and changing its address)
+			while (length > 0) {
+				long toCopy = Math.min(length, COPY_PER_BATCH);
+				UNSAFE.copyMemory(heapMemory, pos, dst, arrayAddress, toCopy);
+				length -= toCopy;
+				pos += toCopy;
+				arrayAddress += toCopy;
+			}
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void put(int index, byte[] src, int offset, int length) {
+		// check the byte array offset and length
+		if ((offset | length | (offset + length) | (src.length - (offset + length))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		long pos = address + index;
+
+		if (index >= 0 && pos <= addressLimit - length) {
+			long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+			while (length > 0) {
+				long toCopy = Math.min(length, COPY_PER_BATCH);
+				UNSAFE.copyMemory(src, arrayAddress, heapMemory, pos, toCopy);
+				length -= toCopy;
+				pos += toCopy;
+				arrayAddress += toCopy;
+			}
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final boolean getBoolean(int index) {
+		return get(index) != 0;
+	}
+
+	public final void putBoolean(int index, boolean value) {
+		put(index, (byte) (value ? 1 : 0));
+	}
+
+	@SuppressWarnings("restriction")
+	public final char getChar(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			return UNSAFE.getChar(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final char getCharLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getChar(index);
+		} else {
+			return Character.reverseBytes(getChar(index));
+		}
+	}
+
+	public final char getCharBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Character.reverseBytes(getChar(index));
+		} else {
+			return getChar(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putChar(int index, char value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			UNSAFE.putChar(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putCharLittleEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, value);
+		} else {
+			putChar(index, Character.reverseBytes(value));
+		}
+	}
+
+	public final void putCharBigEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, Character.reverseBytes(value));
+		} else {
+			putChar(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final short getShort(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			return UNSAFE.getShort(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final short getShortLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getShort(index);
+		} else {
+			return Short.reverseBytes(getShort(index));
+		}
+	}
+	
+	public final short getShortBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Short.reverseBytes(getShort(index));
+		} else {
+			return getShort(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putShort(int index, short value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			UNSAFE.putShort(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putShortLittleEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, value);
+		} else {
+			putShort(index, Short.reverseBytes(value));
+		}
+	}
+	
+	public final void putShortBigEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, Short.reverseBytes(value));
+		} else {
+			putShort(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final int getInt(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 4) {
+			return UNSAFE.getInt(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final int getIntLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getInt(index);
+		} else {
+			return Integer.reverseBytes(getInt(index));
+		}
+	}
+	
+	public final int getIntBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Integer.reverseBytes(getInt(index));
+		} else {
+			return getInt(index);
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void putInt(int index, int value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 4) {
+			UNSAFE.putInt(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putIntLittleEndian(int index, int value) {
+		if (LITTLE_ENDIAN) {
+			putInt(index, value);
+		} else {
+			putInt(index, Integer.reverseBytes(value));
+		}
+	}
+	
+	public final void putIntBigEndian(int index, int value) {
+		if (LITTLE_ENDIAN) {
+			putInt(index, Integer.reverseBytes(value));
+		} else {
+			putInt(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final long getLong(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 8) {
+			return UNSAFE.getLong(heapMemory, pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final long getLongLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getLong(index);
+		} else {
+			return Long.reverseBytes(getLong(index));
+		}
+	}
+	
+	public final long getLongBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Long.reverseBytes(getLong(index));
+		} else {
+			return getLong(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putLong(int index, long value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 8) {
+			UNSAFE.putLong(heapMemory, pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putLongLittleEndian(int index, long value) {
+		if (LITTLE_ENDIAN) {
+			putLong(index, value);
+		} else {
+			putLong(index, Long.reverseBytes(value));
+		}
+	}
+	
+	public final void putLongBigEndian(int index, long value) {
+		if (LITTLE_ENDIAN) {
+			putLong(index, Long.reverseBytes(value));
+		} else {
+			putLong(index, value);
+		}
+	}
+
+	public final float getFloat(int index) {
+		return Float.intBitsToFloat(getInt(index));
+	}
+	
+	public final float getFloatLittleEndian(int index) {
+		return Float.intBitsToFloat(getIntLittleEndian(index));
+	}
+	
+	public final float getFloatBigEndian(int index) {
+		return Float.intBitsToFloat(getIntBigEndian(index));
+	}
+	
+	public final void putFloat(int index, float value) {
+		putInt(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final void putFloatLittleEndian(int index, float value) {
+		putIntLittleEndian(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final void putFloatBigEndian(int index, float value) {
+		putIntBigEndian(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final double getDouble(int index) {
+		return Double.longBitsToDouble(getLong(index));
+	}
+	
+	public final double getDoubleLittleEndian(int index) {
+		return Double.longBitsToDouble(getLongLittleEndian(index));
+	}
+
+	public final double getDoubleBigEndian(int index) {
+		return Double.longBitsToDouble(getLongBigEndian(index));
+	}
+	
+	public final void putDouble(int index, double value) {
+		putLong(index, Double.doubleToRawLongBits(value));
+	}
+	
+	public final void putDoubleLittleEndian(int index, double value) {
+		putLongLittleEndian(index, Double.doubleToRawLongBits(value));
+	}
+	
+	public final void putDoubleBigEndian(int index, double value) {
+		putLongBigEndian(index, Double.doubleToRawLongBits(value));
+	}
+
+	// -------------------------------------------------------------------------
+	//                     Bulk Read and Write Methods
+	// -------------------------------------------------------------------------
+
+	public final void get(DataOutput out, int offset, int length) throws IOException {
+		if (heapMemory != null) {
+			out.write(heapMemory, offset, length);
+		}
+		else {
+			while (length >= 8) {
+				out.writeLong(getLongBigEndian(offset));
+				offset += 8;
+				length -= 8;
+			}
+	
+			while (length > 0) {
+				out.writeByte(get(offset));
+				offset++;
+				length--;
+			}
+		}
+	}
+
+	public final void put(DataInput in, int offset, int length) throws IOException {
+		if (heapMemory != null) {
+			in.readFully(heapMemory, offset, length);
+		}
+		else {
+			while (length >= 8) {
+				putLongBigEndian(offset, in.readLong());
+				offset += 8;
+				length -= 8;
+			}
+			while(length > 0) {
+				put(offset, in.readByte());
+				offset++;
+				length--;
+			}
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void get(int offset, ByteBuffer target, int numBytes) {
+		if (heapMemory != null) {
+			// ByteBuffer performs the boundary checks
+			target.put(heapMemory, offset, numBytes);
+		}
+		else {
+			// check the byte array offset and length
+			if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+				throw new IndexOutOfBoundsException();
+			}
+	
+			final int targetOffset = target.position();
+			final int remaining = target.remaining();
+	
+			if (remaining < numBytes) {
+				throw new BufferOverflowException();
+			}
+	
+			if (target.isDirect()) {
+				// copy to the target memory directly
+				final long targetPointer = getAddress(target) + targetOffset;
+				final long sourcePointer = address + offset;
+	
+				if (sourcePointer <= addressLimit - numBytes) {
+					UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+				}
+				else if (address > addressLimit) {
+					throw new IllegalStateException("This segment has been freed.");
+				}
+				else {
+					throw new IndexOutOfBoundsException();
+				}
+			}
+			else if (target.hasArray()) {
+				// move directly into the byte array
+				get(offset, target.array(), targetOffset + target.arrayOffset(), numBytes);
+	
+				// this must be after the get() call to ensue that the byte buffer is not
+				// modified in case the call fails
+				target.position(targetOffset + numBytes);
+			}
+			else {
+				// neither heap buffer nor direct buffer
+				while (target.hasRemaining()) {
+					target.put(get(offset++));
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void put(int offset, ByteBuffer source, int numBytes) {
+		if (heapMemory != null) {
+			source.get(heapMemory, offset, numBytes);
+		}
+		else {
+			// check the byte array offset and length
+			if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+				throw new IndexOutOfBoundsException();
+			}
+	
+			final int sourceOffset = source.position();
+			final int remaining = source.remaining();
+	
+			if (remaining < numBytes) {
+				throw new BufferUnderflowException();
+			}
+	
+			if (source.isDirect()) {
+				// copy to the target memory directly
+				final long sourcePointer = getAddress(source) + sourceOffset;
+				final long targetPointer = address + offset;
+	
+				if (sourcePointer <= addressLimit - numBytes) {
+					UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+				}
+				else if (address > addressLimit) {
+					throw new IllegalStateException("This segment has been freed.");
+				}
+				else {
+					throw new IndexOutOfBoundsException();
+				}
+			}
+			else if (source.hasArray()) {
+				// move directly into the byte array
+				put(offset, source.array(), sourceOffset + source.arrayOffset(), numBytes);
+	
+				// this must be after the get() call to ensue that the byte buffer is not
+				// modified in case the call fails
+				source.position(sourceOffset + numBytes);
+			}
+			else {
+				// neither heap buffer nor direct buffer
+				while (source.hasRemaining()) {
+					put(offset++, source.get());
+				}
+			}
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void copyTo(int offset, PureHybridMemorySegment target, int targetOffset, int numBytes) {
+		final byte[] thisHeapRef = this.heapMemory;
+		final byte[] otherHeapRef = target.heapMemory;
+		final long thisPointer = this.address + offset;
+		final long otherPointer = target.address + targetOffset;
+
+		if (numBytes >= 0 & thisPointer <= this.addressLimit - numBytes & otherPointer <= target.addressLimit - numBytes) {
+			UNSAFE.copyMemory(thisHeapRef, thisPointer, otherHeapRef, otherPointer, numBytes);
+		}
+		else if (address > addressLimit | target.address > target.addressLimit) {
+			throw new IllegalStateException("segment has been freed.");
+		}
+		else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public int compare(PureHybridMemorySegment seg2, int offset1, int offset2, int len) {
+		while (len >= 8) {
+			long l1 = this.getLongBigEndian(offset1);
+			long l2 = seg2.getLongBigEndian(offset2);
+
+			if (l1 != l2) {
+				return (l1 < l2) ^ (l1 < 0) ^ (l2 < 0) ? -1 : 1;
+			}
+
+			offset1 += 8;
+			offset2 += 8;
+			len -= 8;
+		}
+		while (len > 0) {
+			int b1 = this.get(offset1) & 0xff;
+			int b2 = seg2.get(offset2) & 0xff;
+			int cmp = b1 - b2;
+			if (cmp != 0) {
+				return cmp;
+			}
+			offset1++;
+			offset2++;
+			len--;
+		}
+		return 0;
+	}
+
+	public void swapBytes(byte[] tempBuffer, PureHybridMemorySegment seg2, int offset1, int offset2, int len) {
+		if (len < 32) {
+			// fast path for short copies
+			while (len >= 8) {
+				long tmp = this.getLong(offset1);
+				this.putLong(offset1, seg2.getLong(offset2));
+				seg2.putLong(offset2, tmp);
+				offset1 += 8;
+				offset2 += 8;
+				len -= 8;
+			}
+			while (len > 0) {
+				byte tmp = this.get(offset1);
+				this.put(offset1, seg2.get(offset2));
+				seg2.put(offset2, tmp);
+				offset1++;
+				offset2++;
+				len--;
+			}
+		}
+		else if ( (offset1 | offset2 | len | (offset1 + len) | (offset2 + len) |
+				(this.size - (offset1 + len)) | (seg2.size() - (offset2 + len))) < 0 || len > tempBuffer.length)
+		{
+			throw new IndexOutOfBoundsException();
+		}
+		else {
+			final long thisPos = this.address + offset1;
+			final long otherPos = seg2.address + offset2;
+
+			if (thisPos <= this.addressLimit - len && otherPos <= seg2.addressLimit - len) {
+				final long arrayAddress = BYTE_ARRAY_BASE_OFFSET;
+
+				// this -> temp buffer
+				UNSAFE.copyMemory(this.heapMemory, thisPos, tempBuffer, arrayAddress, len);
+
+				// other -> this
+				UNSAFE.copyMemory(seg2.heapMemory, otherPos, this.heapMemory, thisPos, len);
+
+				// temp buffer -> other
+				UNSAFE.copyMemory(tempBuffer, arrayAddress, seg2.heapMemory, otherPos, len);
+			}
+			else if (this.address <= 0 || seg2.address <= 0) {
+				throw new IllegalStateException("Memory segment has been freed.");
+			}
+			else {
+				// index is in fact invalid
+				throw new IndexOutOfBoundsException();
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                     Utilities for native memory accesses and checks
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("restriction")
+	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+	@SuppressWarnings("restriction")
+	private static final long BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+	private static final long COPY_PER_BATCH = 1024 * 1024;
+
+	private static final Field ADDRESS_FIELD;
+
+	static {
+		try {
+			ADDRESS_FIELD = java.nio.Buffer.class.getDeclaredField("address");
+			ADDRESS_FIELD.setAccessible(true);
+		}
+		catch (Throwable t) {
+			throw new RuntimeException("Cannot initialize DirectMemorySegment - direct memory not supported by the JVM.");
+		}
+	}
+
+	private static long getAddress(ByteBuffer buf) {
+		try {
+			return (Long) ADDRESS_FIELD.get(buf);
+		}
+		catch (Throwable t) {
+			throw new RuntimeException("Could not access direct byte buffer address.", t);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java
new file mode 100644
index 0000000..cda48e1
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java
@@ -0,0 +1,359 @@
+/*
+ * 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 java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class PureHybridMemorySegmentOutView implements DataOutputView {
+
+	private PureHybridMemorySegment 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<PureHybridMemorySegment> memorySource;
+	
+	private final List<PureHybridMemorySegment> fullSegments;
+	
+
+	private byte[] utfBuffer;		// the reusable array for UTF encodings
+
+
+	public PureHybridMemorySegmentOutView(List<PureHybridMemorySegment> emptySegments,
+										  List<PureHybridMemorySegment> 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 PureHybridMemorySegment nextSegment(PureHybridMemorySegment current, int positionInCurrent) throws EOFException {
+		int size = this.memorySource.size();
+		if (size > 0) {
+			final PureHybridMemorySegment next = this.memorySource.remove(size - 1);
+			this.fullSegments.add(next);
+			return next;
+		} else {
+			throw new EOFException();
+		}
+	}
+	
+	public PureHybridMemorySegment 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(PureHybridMemorySegment 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/PureOffHeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java
new file mode 100644
index 0000000..1280242
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java
@@ -0,0 +1,790 @@
+/*
+ * 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.MemorySegment;
+import org.apache.flink.core.memory.MemoryUtils;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.BufferOverflowException;
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+public final class PureOffHeapMemorySegment {
+
+	/** Constant that flags the byte order. Because this is a boolean constant,
+	 * the JIT compiler can use this well to aggressively eliminate the non-applicable code paths */
+	private static final boolean LITTLE_ENDIAN = (ByteOrder.nativeOrder() == ByteOrder.LITTLE_ENDIAN);
+	
+	/** The direct byte buffer that allocated the memory */
+	private ByteBuffer buffer;
+
+	/** The address to the off-heap data */
+	private long address;
+
+	/** The address one byte after the last addressable byte.
+	 *  This is address + size while the segment is not disposed */
+	private final long addressLimit;
+
+	/** The size in bytes of the memory segment */
+	private final int size;
+
+	// -------------------------------------------------------------------------
+	//                             Constructors
+	// -------------------------------------------------------------------------
+
+	/**
+	 * Creates a new memory segment that represents the memory backing the given direct byte buffer.
+	 * Note that the given ByteBuffer must be direct {@link java.nio.ByteBuffer#allocateDirect(int)},
+	 * otherwise this method with throw an IllegalArgumentException. data in the given byte array.
+	 *
+	 * @param buffer The byte buffer whose memory is represented by this memory segment.
+	 * @throws IllegalArgumentException Thrown, if the given ByteBuffer is not direct.
+	 */
+	public PureOffHeapMemorySegment(ByteBuffer buffer) {
+		if (buffer == null || !buffer.isDirect()) {
+			throw new IllegalArgumentException("Can't initialize from non-direct ByteBuffer.");
+		}
+
+		this.buffer = buffer;
+		this.size = buffer.capacity();
+		this.address = getAddress(buffer);
+		this.addressLimit = this.address + size;
+
+		if (address >= Long.MAX_VALUE - Integer.MAX_VALUE) {
+			throw new RuntimeException("Segment initialized with too large address: " + address);
+		}
+	}
+
+	// -------------------------------------------------------------------------
+	//                      Direct Memory Segment Specifics
+	// -------------------------------------------------------------------------
+
+	/**
+	 * Gets the buffer that owns the memory of this memory segment.
+	 *
+	 * @return The byte buffer that owns the memory of this memory segment.
+	 */
+	public ByteBuffer getBuffer() {
+		return this.buffer;
+	}
+
+	/**
+	 * Gets the memory address of the memory backing this memory segment.
+	 *
+	 * @return The memory start address of the memory backing this memory segment. 
+	 */
+	public long getAddress() {
+		return address;
+	}
+
+	// -------------------------------------------------------------------------
+	//                        MemorySegment Accessors
+	// -------------------------------------------------------------------------
+	
+	public final boolean isFreed() {
+		return this.address > this.addressLimit;
+	}
+	
+	public final void free() {
+		// this ensures we can place no more data and trigger
+		// the checks for the freed segment
+		this.address = this.addressLimit + 1;
+		this.buffer = null;
+	}
+	
+	public final int size() {
+		return this.size;
+	}
+
+	public ByteBuffer wrap(int offset, int length) {
+		if (offset < 0 || offset > this.size || offset > this.size - length) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		this.buffer.limit(offset + length);
+		this.buffer.position(offset);
+
+		return this.buffer;
+	}
+
+
+	// ------------------------------------------------------------------------
+	//                    Random Access get() and put() methods
+	// ------------------------------------------------------------------------
+	
+	@SuppressWarnings("restriction")
+	public final byte get(int index) {
+
+		final long pos = address + index;
+		if (index >= 0 && pos < addressLimit) {
+			return UNSAFE.getByte(pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void put(int index, byte b) {
+
+		final long pos = address + index;
+		if (index >= 0 && pos < addressLimit) {
+			UNSAFE.putByte(pos, b);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+	
+	public final void get(int index, byte[] dst) {
+		get(index, dst, 0, dst.length);
+	}
+	
+	public final void put(int index, byte[] src) {
+		put(index, src, 0, src.length);
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void get(int index, byte[] dst, int offset, int length) {
+
+		// check the byte array offset and length
+		if ((offset | length | (offset + length) | (dst.length - (offset + length))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		long pos = address + index;
+
+		if (index >= 0 && pos <= addressLimit - length) {
+			long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+
+			// the copy must proceed in batches not too large, because the JVM may
+			// poll for points that are safe for GC (moving the array and changing its address)
+			while (length > 0) {
+				long toCopy = (length > COPY_PER_BATCH) ? COPY_PER_BATCH : length;
+				UNSAFE.copyMemory(null, pos, dst, arrayAddress, toCopy);
+				length -= toCopy;
+				pos += toCopy;
+				arrayAddress += toCopy;
+			}
+		}
+		else if (address <= 0) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void put(int index, byte[] src, int offset, int length) {
+		// check the byte array offset and length
+		if ((offset | length | (offset + length) | (src.length - (offset + length))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		long pos = address + index;
+
+		if (index >= 0 && pos <= addressLimit - length) {
+
+			long arrayAddress = BYTE_ARRAY_BASE_OFFSET + offset;
+			while (length > 0) {
+				long toCopy = (length > COPY_PER_BATCH) ? COPY_PER_BATCH : length;
+				UNSAFE.copyMemory(src, arrayAddress, null, pos, toCopy);
+				length -= toCopy;
+				pos += toCopy;
+				arrayAddress += toCopy;
+			}
+		}
+		else if (address <= 0) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final boolean getBoolean(int index) {
+		return get(index) != 0;
+	}
+
+	public final void putBoolean(int index, boolean value) {
+		put(index, (byte) (value ? 1 : 0));
+	}
+
+	@SuppressWarnings("restriction")
+	public final char getChar(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			return UNSAFE.getChar(pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final char getCharLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getChar(index);
+		} else {
+			return Character.reverseBytes(getChar(index));
+		}
+	}
+
+	public final char getCharBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Character.reverseBytes(getChar(index));
+		} else {
+			return getChar(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putChar(int index, char value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			UNSAFE.putChar(pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putCharLittleEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, value);
+		} else {
+			putChar(index, Character.reverseBytes(value));
+		}
+	}
+
+	public final void putCharBigEndian(int index, char value) {
+		if (LITTLE_ENDIAN) {
+			putChar(index, Character.reverseBytes(value));
+		} else {
+			putChar(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final short getShort(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			return UNSAFE.getShort(pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final short getShortLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getShort(index);
+		} else {
+			return Short.reverseBytes(getShort(index));
+		}
+	}
+	
+	public final short getShortBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Short.reverseBytes(getShort(index));
+		} else {
+			return getShort(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putShort(int index, short value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 2) {
+			UNSAFE.putShort(pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putShortLittleEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, value);
+		} else {
+			putShort(index, Short.reverseBytes(value));
+		}
+	}
+	
+	public final void putShortBigEndian(int index, short value) {
+		if (LITTLE_ENDIAN) {
+			putShort(index, Short.reverseBytes(value));
+		} else {
+			putShort(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final int getInt(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 4) {
+			return UNSAFE.getInt(pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final int getIntLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getInt(index);
+		} else {
+			return Integer.reverseBytes(getInt(index));
+		}
+	}
+	
+	public final int getIntBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Integer.reverseBytes(getInt(index));
+		} else {
+			return getInt(index);
+		}
+	}
+	
+	@SuppressWarnings("restriction")
+	public final void putInt(int index, int value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 4) {
+			UNSAFE.putInt(pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putIntLittleEndian(int index, int value) {
+		if (LITTLE_ENDIAN) {
+			putInt(index, value);
+		} else {
+			putInt(index, Integer.reverseBytes(value));
+		}
+	}
+	
+	public final void putIntBigEndian(int index, int value) {
+		if (LITTLE_ENDIAN) {
+			putInt(index, Integer.reverseBytes(value));
+		} else {
+			putInt(index, value);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final long getLong(int index) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 8) {
+			return UNSAFE.getLong(pos);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final long getLongLittleEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return getLong(index);
+		} else {
+			return Long.reverseBytes(getLong(index));
+		}
+	}
+	
+	public final long getLongBigEndian(int index) {
+		if (LITTLE_ENDIAN) {
+			return Long.reverseBytes(getLong(index));
+		} else {
+			return getLong(index);
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void putLong(int index, long value) {
+		final long pos = address + index;
+		if (index >= 0 && pos <= addressLimit - 8) {
+			UNSAFE.putLong(pos, value);
+		}
+		else if (address > addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			// index is in fact invalid
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public final void putLongLittleEndian(int index, long value) {
+		if (LITTLE_ENDIAN) {
+			putLong(index, value);
+		} else {
+			putLong(index, Long.reverseBytes(value));
+		}
+	}
+	
+	public final void putLongBigEndian(int index, long value) {
+		if (LITTLE_ENDIAN) {
+			putLong(index, Long.reverseBytes(value));
+		} else {
+			putLong(index, value);
+		}
+	}
+
+	public final float getFloat(int index) {
+		return Float.intBitsToFloat(getInt(index));
+	}
+	
+	public final float getFloatLittleEndian(int index) {
+		return Float.intBitsToFloat(getIntLittleEndian(index));
+	}
+	
+	public final float getFloatBigEndian(int index) {
+		return Float.intBitsToFloat(getIntBigEndian(index));
+	}
+	
+	public final void putFloat(int index, float value) {
+		putInt(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final void putFloatLittleEndian(int index, float value) {
+		putIntLittleEndian(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final void putFloatBigEndian(int index, float value) {
+		putIntBigEndian(index, Float.floatToRawIntBits(value));
+	}
+	
+	public final double getDouble(int index) {
+		return Double.longBitsToDouble(getLong(index));
+	}
+	
+	public final double getDoubleLittleEndian(int index) {
+		return Double.longBitsToDouble(getLongLittleEndian(index));
+	}
+
+	public final double getDoubleBigEndian(int index) {
+		return Double.longBitsToDouble(getLongBigEndian(index));
+	}
+	
+	public final void putDouble(int index, double value) {
+		putLong(index, Double.doubleToRawLongBits(value));
+	}
+	
+	public final void putDoubleLittleEndian(int index, double value) {
+		putLongLittleEndian(index, Double.doubleToRawLongBits(value));
+	}
+	
+	public final void putDoubleBigEndian(int index, double value) {
+		putLongBigEndian(index, Double.doubleToRawLongBits(value));
+	}
+
+	// -------------------------------------------------------------------------
+	//                     Bulk Read and Write Methods
+	// -------------------------------------------------------------------------
+
+	public final void get(DataOutput out, int offset, int length) throws IOException {
+		while (length >= 8) {
+			out.writeLong(getLongBigEndian(offset));
+			offset += 8;
+			length -= 8;
+		}
+
+		while(length > 0) {
+			out.writeByte(get(offset));
+			offset++;
+			length--;
+		}
+	}
+
+	public final void put(DataInput in, int offset, int length) throws IOException {
+		while (length >= 8) {
+			putLongBigEndian(offset, in.readLong());
+			offset += 8;
+			length -= 8;
+		}
+		while(length > 0) {
+			put(offset, in.readByte());
+			offset++;
+			length--;
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void get(int offset, ByteBuffer target, int numBytes) {
+
+		// check the byte array offset and length
+		if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		final int targetOffset = target.position();
+		final int remaining = target.remaining();
+
+		if (remaining < numBytes) {
+			throw new BufferOverflowException();
+		}
+
+		if (target.isDirect()) {
+			// copy to the target memory directly
+			final long targetPointer = getAddress(target) + targetOffset;
+			final long sourcePointer = address + offset;
+
+			if (sourcePointer <= addressLimit - numBytes) {
+				UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+			}
+			else if (address > addressLimit) {
+				throw new IllegalStateException("This segment has been freed.");
+			}
+			else {
+				throw new IndexOutOfBoundsException();
+			}
+		}
+		else if (target.hasArray()) {
+			// move directly into the byte array
+			get(offset, target.array(), targetOffset + target.arrayOffset(), numBytes);
+
+			// this must be after the get() call to ensue that the byte buffer is not
+			// modified in case the call fails
+			target.position(targetOffset + numBytes);
+		}
+		else {
+			// neither heap buffer nor direct buffer
+			while (target.hasRemaining()) {
+				target.put(get(offset++));
+			}
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void put(int offset, ByteBuffer source, int numBytes) {
+
+		// check the byte array offset and length
+		if ((offset | numBytes | (offset + numBytes) | (size - (offset + numBytes))) < 0) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		final int sourceOffset = source.position();
+		final int remaining = source.remaining();
+
+		if (remaining < numBytes) {
+			throw new BufferUnderflowException();
+		}
+
+		if (source.isDirect()) {
+			// copy to the target memory directly
+			final long sourcePointer = getAddress(source) + sourceOffset;
+			final long targetPointer = address + offset;
+
+			if (sourcePointer <= addressLimit - numBytes) {
+				UNSAFE.copyMemory(sourcePointer, targetPointer, numBytes);
+			}
+			else if (address > addressLimit) {
+				throw new IllegalStateException("This segment has been freed.");
+			}
+			else {
+				throw new IndexOutOfBoundsException();
+			}
+		}
+		else if (source.hasArray()) {
+			// move directly into the byte array
+			put(offset, source.array(), sourceOffset + source.arrayOffset(), numBytes);
+
+			// this must be after the get() call to ensue that the byte buffer is not
+			// modified in case the call fails
+			source.position(sourceOffset + numBytes);
+		}
+		else {
+			// neither heap buffer nor direct buffer
+			while (source.hasRemaining()) {
+				put(offset++, source.get());
+			}
+		}
+	}
+
+	@SuppressWarnings("restriction")
+	public final void copyTo(int offset, PureOffHeapMemorySegment target, int targetOffset, int numBytes) {
+		final long thisPointer = address + offset;
+		final long otherPointer = target.address + targetOffset;
+
+		if (numBytes >= 0 && thisPointer <= addressLimit - numBytes && otherPointer <= target.addressLimit - numBytes) {
+			UNSAFE.copyMemory(thisPointer, otherPointer, numBytes);
+		}
+		else if (address > addressLimit || target.address > target.addressLimit) {
+			throw new IllegalStateException("This segment has been freed.");
+		}
+		else {
+			throw new IndexOutOfBoundsException();
+		}
+	}
+
+	public int compare(MemorySegment seg2, int offset1, int offset2, int len) {
+		while (len >= 8) {
+			long l1 = this.getLongBigEndian(offset1);
+			long l2 = seg2.getLongBigEndian(offset2);
+
+			if (l1 != l2) {
+				return (l1 < l2) ^ (l1 < 0) ^ (l2 < 0) ? -1 : 1;
+			}
+
+			offset1 += 8;
+			offset2 += 8;
+			len -= 8;
+		}
+		while (len > 0) {
+			int b1 = this.get(offset1) & 0xff;
+			int b2 = seg2.get(offset2) & 0xff;
+			int cmp = b1 - b2;
+			if (cmp != 0) {
+				return cmp;
+			}
+			offset1++;
+			offset2++;
+			len--;
+		}
+		return 0;
+	}
+
+	public void swapBytes(byte[] tempBuffer, PureOffHeapMemorySegment seg2, int offset1, int offset2, int len) {
+		if (len < 32) {
+			// fast path for short copies
+			while (len >= 8) {
+				long tmp = this.getLong(offset1);
+				this.putLong(offset1, seg2.getLong(offset2));
+				seg2.putLong(offset2, tmp);
+				offset1 += 8;
+				offset2 += 8;
+				len -= 8;
+			}
+			while (len > 0) {
+				byte tmp = this.get(offset1);
+				this.put(offset1, seg2.get(offset2));
+				seg2.put(offset2, tmp);
+				offset1++;
+				offset2++;
+				len--;
+			}
+		}
+		else if ( (offset1 | offset2 | len | (offset1 + len) | (offset2 + len) |
+				(this.size - (offset1 + len)) | (seg2.size() - (offset2 + len))) < 0 || len > tempBuffer.length)
+		{
+			throw new IndexOutOfBoundsException();
+		}
+		else {
+			final long thisPos = this.address + offset1;
+			final long otherPos = seg2.address + offset2;
+
+			if (thisPos <= this.addressLimit - len && otherPos <= seg2.addressLimit - len) {
+				final long arrayAddress = BYTE_ARRAY_BASE_OFFSET;
+
+				// this -> temp buffer
+				UNSAFE.copyMemory(null, thisPos, tempBuffer, arrayAddress, len);
+
+				// other -> this
+				UNSAFE.copyMemory(null, otherPos, null, thisPos, len);
+
+				// temp buffer -> other
+				UNSAFE.copyMemory(tempBuffer, arrayAddress, null, otherPos, len);
+			}
+			else if (this.address <= 0 || seg2.address <= 0) {
+				throw new IllegalStateException("Memory segment has been freed.");
+			}
+			else {
+				// index is in fact invalid
+				throw new IndexOutOfBoundsException();
+			}
+		}
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                     Utilities for native memory accesses and checks
+	// --------------------------------------------------------------------------------------------
+
+	@SuppressWarnings("restriction")
+	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
+
+	@SuppressWarnings("restriction")
+	private static final long BYTE_ARRAY_BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
+
+	private static final long COPY_PER_BATCH = 1024 * 1024;
+
+	private static final Field ADDRESS_FIELD;
+
+	static {
+		try {
+			ADDRESS_FIELD = java.nio.Buffer.class.getDeclaredField("address");
+			ADDRESS_FIELD.setAccessible(true);
+		}
+		catch (Throwable t) {
+			throw new RuntimeException("Cannot initialize DirectMemorySegment - direct memory not supported by the JVM.");
+		}
+	}
+
+	private static long getAddress(ByteBuffer buf) {
+		try {
+			return (Long) ADDRESS_FIELD.get(buf);
+		}
+		catch (Throwable t) {
+			throw new RuntimeException("Could not access direct byte buffer address.", t);
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java
new file mode 100644
index 0000000..d898dee
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java
@@ -0,0 +1,359 @@
+/*
+ * 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 java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+import java.util.List;
+
+public final class PureOffHeapMemorySegmentOutView implements DataOutputView {
+
+	private PureOffHeapMemorySegment 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<PureOffHeapMemorySegment> memorySource;
+	
+	private final List<PureOffHeapMemorySegment> fullSegments;
+	
+
+	private byte[] utfBuffer;		// the reusable array for UTF encodings
+
+
+	public PureOffHeapMemorySegmentOutView(List<PureOffHeapMemorySegment> emptySegments,
+										   List<PureOffHeapMemorySegment> 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 PureOffHeapMemorySegment nextSegment(PureOffHeapMemorySegment current, int positionInCurrent) throws EOFException {
+		int size = this.memorySource.size();
+		if (size > 0) {
+			final PureOffHeapMemorySegment next = this.memorySource.remove(size - 1);
+			this.fullSegments.add(next);
+			return next;
+		} else {
+			throw new EOFException();
+		}
+	}
+	
+	public PureOffHeapMemorySegment 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(PureOffHeapMemorySegment 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/StringSerializationSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java
new file mode 100644
index 0000000..2163eb5
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java
@@ -0,0 +1,207 @@
+/*
+ * 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.StringSerializer;
+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 StringSerializationSpeedBenchmark {
+	
+	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);
+
+		final String[] randomStrings = generateRandomStrings(5468917685263896L, 1000, 128, 6, true);
+
+		final StringSerializer ser = StringSerializer.INSTANCE;
+		
+		final int outerRounds = 10;
+		final int innerRounds = 5000;
+
+		{
+			System.out.println("testing core heap memory segment");
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+
+				ArrayList<MemorySegment> memory = new ArrayList<>();
+				memory.add(HeapMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null));
+				ArrayList<MemorySegment> target = new ArrayList<>();
+
+				CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+				for (int i = 0; i < innerRounds; i++) {
+					for (String s : randomStrings) {
+						ser.serialize(s, output);
+					}
+				}
+			}
+			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 memory segment on heap");
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+
+				ArrayList<MemorySegment> memory = new ArrayList<>();
+				memory.add(HybridMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null));
+				ArrayList<MemorySegment> target = new ArrayList<>();
+
+				CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+				for (int i = 0; i < innerRounds; i++) {
+					for (String s : randomStrings) {
+						ser.serialize(s, output);
+					}
+				}
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Core hybrid memory segment on heap took %,d msecs", (stop - start) / 1000000));
+		}
+
+		{
+			System.out.println("testing core hybrid memory segment off heap");
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+
+				ArrayList<MemorySegment> memory = new ArrayList<>();
+				memory.add(HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(largeOffHeap, null));
+				ArrayList<MemorySegment> target = new ArrayList<>();
+
+				CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+
+				for (int i = 0; i < innerRounds; i++) {
+					for (String s : randomStrings) {
+						ser.serialize(s, output);
+					}
+				}
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Core hybrid memory segment off heap took %,d msecs", (stop - start) / 1000000));
+		}
+		
+		{
+			System.out.println("testing pure hybrid memory segment on heap");
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+			
+				ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+				memory.add(new PureHybridMemorySegment(largeSegment));
+				ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+	
+				PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+					
+				for (int i = 0; i < innerRounds; i++) {
+					for (String s : randomStrings) {
+						ser.serialize(s, output);
+					}
+				}
+			}
+			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");
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+				
+				ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+				memory.add(new PureHybridMemorySegment(largeOffHeap));
+				ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+	
+				PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+				
+				for (int i = 0; i < innerRounds; i++) {
+					for (String s : randomStrings) {
+						ser.serialize(s, output);
+					}
+				}
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Pure hybrid off heap memory segment took %,d msecs", (stop - start) / 1000000));
+		}
+		
+		{
+			System.out.println("testing pure heap memory segment");
+
+			long start = System.nanoTime();
+			for (int outer = 0; outer < outerRounds; outer++) {
+					
+				ArrayList<PureHeapMemorySegment> memory = new ArrayList<>();
+				memory.add(new PureHeapMemorySegment(largeSegment));
+				ArrayList<PureHeapMemorySegment> target = new ArrayList<>();
+				
+				PureHeapMemorySegmentOutView output = new PureHeapMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+				
+				for (int i = 0; i < innerRounds; i++) {
+					for (String s : randomStrings) {
+						ser.serialize(s, output);
+					}
+				}
+			}
+			long stop = System.nanoTime();
+
+			System.out.println(String.format("Pure 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;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java b/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
index b39fb2e..d6c15d9 100644
--- a/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
+++ b/flink-core/src/test/java/org/apache/flink/types/NormalizableKeyTest.java
@@ -16,18 +16,12 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.types;
 
-import org.junit.Assert;
-
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.types.CharValue;
-import org.apache.flink.types.IntValue;
-import org.apache.flink.types.LongValue;
-import org.apache.flink.types.NormalizableKey;
-import org.apache.flink.types.NullValue;
-import org.apache.flink.types.StringValue;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+
+import org.junit.Assert;
 import org.junit.Test;
 
 public class NormalizableKeyTest {
@@ -124,9 +118,9 @@ public class NormalizableKeyTest {
 		
 		for (int i = 0; i < 5; i++) {
 			// self checks
-			for (int x = 0; x < allChars.length; x++) {
-				for (int y = 0; y < allChars.length; y++) {
-					assertNormalizableKey(allChars[x], allChars[y], i);
+			for (CharValue allChar1 : allChars) {
+				for (CharValue allChar : allChars) {
+					assertNormalizableKey(allChar1, allChar, i);
 				}
 			}
 		}
@@ -135,8 +129,8 @@ public class NormalizableKeyTest {
 	@SuppressWarnings("unchecked")
 	private <T extends Comparable<T>> void assertNormalizableKey(NormalizableKey<T> key1, NormalizableKey<T> key2, int len) {
 		
-		byte[] normalizedKeys = new byte[2*len];
-		MemorySegment wrapper = new MemorySegment(normalizedKeys);
+		byte[] normalizedKeys = new byte[32];
+		MemorySegment wrapper = MemorySegmentFactory.wrap(normalizedKeys);
 		
 		key1.copyNormalizedKey(wrapper, 0, len);
 		key2.copyNormalizedKey(wrapper, len, len);
@@ -147,13 +141,13 @@ public class NormalizableKeyTest {
 			int normKey2 = normalizedKeys[len + i] & 0xFF;
 			
 			if ((comp = (normKey1 - normKey2)) != 0) {
-				if (Math.signum(((T) key1).compareTo((T) key2)) != Math.signum(comp)) {
+				if (Math.signum(key1.compareTo((T) key2)) != Math.signum(comp)) {
 					Assert.fail("Normalized key comparison differs from actual key comparision");
 				}
 				return;
 			}
 		}
-		if (((T) key1).compareTo((T) key2) != 0 && key1.getMaxNormalizedKeyLen() <= len) {
+		if (key1.compareTo((T) key2) != 0 && key1.getMaxNormalizedKeyLen() <= len) {
 			Assert.fail("Normalized key was not able to distinguish keys, " +
 					"although it should as the length of it sufficies to uniquely identify them");
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
index c742ce5..87d89eb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/execution/Environment.java
@@ -29,7 +29,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.api.common.JobID;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
index 736c245..2ca7f78 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelInputView.java
@@ -27,8 +27,8 @@ import java.util.List;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.MathUtils;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
index b6c500f..8415d09 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/FileChannelOutputView.java
@@ -26,8 +26,8 @@ import java.util.List;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.MemoryManager;
 
 /**
  * A {@link org.apache.flink.core.memory.DataOutputView} that is backed by a {@link BlockChannelWriter}, making it effectively a data output

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
index 718b8af..23dccb0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessInputView.java
@@ -23,7 +23,7 @@ import java.util.ArrayList;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.SeekableDataInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
 import org.apache.flink.runtime.util.MathUtils;
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
index 6ed9f4a..427fe84 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/RandomAccessOutputView.java
@@ -23,7 +23,7 @@ import java.io.EOFException;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.SeekableDataOutputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.runtime.util.MathUtils;
 
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
index 7d8d485..355b2eb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SeekableFileChannelInputView.java
@@ -28,8 +28,8 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.MathUtils;
 
 /**


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java
deleted file mode 100644
index 23081c9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedInputView.java
+++ /dev/null
@@ -1,566 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.UTFDataFormatException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-
-/**
- * The base class for all input views that are backed by multiple memory pages. This base class contains all
- * decoding methods to read data from a page and detect page boundary crossing. The concrete sub classes must
- * implement the methods to provide the next memory page once the boundary is crossed.
- */
-public abstract class AbstractPagedInputView implements DataInputView {
-	
-	private MemorySegment currentSegment;
-	
-	protected final int headerLength;				// the number of bytes to skip at the beginning of each segment
-	
-	private int positionInSegment;					// the offset in the current segment
-	
-	private int limitInSegment;						// the limit in the current segment before switching to the next
-	
-	private byte[] utfByteBuffer;					// reusable byte buffer for utf-8 decoding
-	private char[] utfCharBuffer;					// reusable char buffer for utf-8 decoding
-	
-	
-	// --------------------------------------------------------------------------------------------
-	//                                    Constructors
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Creates a new view that starts with the given segment. The input starts directly after the header
-	 * of the given page. If the header size is zero, it starts at the beginning. The specified initial
-	 * limit describes up to which position data may be read from the current segment, before the view must
-	 * advance to the next segment.
-	 * 
-	 * @param initialSegment The memory segment to start reading from.
-	 * @param initialLimit The position one after the last valid byte in the initial segment.
-	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header. This
-	 *                     length must be the same for all memory segments.
-	 */
-	protected AbstractPagedInputView(MemorySegment initialSegment, int initialLimit, int headerLength) {
-		this.headerLength = headerLength;
-		this.positionInSegment = headerLength;
-		seekInput(initialSegment, headerLength, initialLimit);
-	}
-	
-	/**
-	 * Creates a new view that is initially not bound to a memory segment. This constructor is typically
-	 * for views that always seek first.
-	 * <p>
-	 * WARNING: The view is not readable until the first call to either {@link #advance()}, 
-	 * or to {@link #seekInput(MemorySegment, int, int)}.
-	 * 
-	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
-	 */
-	protected AbstractPagedInputView(int headerLength) {
-		this.headerLength = headerLength;
-	}
-
-	// --------------------------------------------------------------------------------------------
-	//                                  Page Management
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Gets the memory segment that will be used to read the next bytes from. If the segment is exactly exhausted,
-	 * meaning that the last byte read was the last byte available in the segment, then this segment will
-	 * not serve the next bytes. The segment to serve the next bytes will be obtained through the
-	 * {@link #nextSegment(MemorySegment)} method.
-	 * 
-	 * @return The current memory segment.
-	 */
-	public MemorySegment getCurrentSegment() {
-		return this.currentSegment;
-	}
-	
-	/**
-	 * Gets the position from which the next byte will be read. If that position is equal to the current limit,
-	 * then the next byte will be read from next segment.
-	 * 
-	 * @return The position from which the next byte will be read.
-	 * @see #getCurrentSegmentLimit()
-	 */
-	public int getCurrentPositionInSegment() {
-		return this.positionInSegment;
-	}
-	
-	/**
-	 * Gets the current limit in the memory segment. This value points to the byte one after the last valid byte
-	 * in the memory segment.
-	 * 
-	 * @return The current limit in the memory segment.
-	 * @see #getCurrentPositionInSegment()
-	 */
-	public int getCurrentSegmentLimit() {
-		return this.limitInSegment;
-	}
-	
-	/**
-	 * The method by which concrete subclasses realize page crossing. This method is invoked when the current page
-	 * is exhausted and a new page is required to continue the reading. If no further page is available, this
-	 * method must throw an {@link EOFException}.
-	 *  
-	 * @param current The current page that was read to its limit. May be {@code null}, if this method is
-	 *                invoked for the first time.
-	 * @return The next page from which the reading should continue. May not be {@code null}. If the input is
-	 *         exhausted, an {@link EOFException} must be thrown instead.
-	 *         
-	 * @throws EOFException Thrown, if no further segment is available.
-	 * @throws IOException Thrown, if the method cannot provide the next page due to an I/O related problem.
-	 */
-	protected abstract MemorySegment nextSegment(MemorySegment current) throws EOFException, IOException;
-	
-	/**
-	 * Gets the limit for reading bytes from the given memory segment. This method must return the position
-	 * of the byte after the last valid byte in the given memory segment. When the position returned by this
-	 * method is reached, the view will attempt to switch to the next memory segment.
-	 * 
-	 * @param segment The segment to determine the limit for.
-	 * @return The limit for the given memory segment.
-	 */
-	protected abstract int getLimitForSegment(MemorySegment segment);
-	
-	/**
-	 * Advances the view to the next memory segment. The reading will continue after the header of the next
-	 * segment. This method uses {@link #nextSegment(MemorySegment)} and {@link #getLimitForSegment(MemorySegment)}
-	 * to get the next segment and set its limit.
-	 * 
-	 * @throws IOException Thrown, if the next segment could not be obtained.
-	 * 
-	 * @see #nextSegment(MemorySegment)
-	 * @see #getLimitForSegment(MemorySegment)
-	 */
-	protected final void advance() throws IOException {
-		// note: this code ensures that in case of EOF, we stay at the same position such that
-		// EOF is reproducible (if nextSegment throws a reproducible EOFException)
-		this.currentSegment = nextSegment(this.currentSegment);
-		this.limitInSegment = getLimitForSegment(this.currentSegment);
-		this.positionInSegment = this.headerLength;
-	}
-	
-	/**
-	 * Sets the internal state of the view such that the next bytes will be read from the given memory segment,
-	 * starting at the given position. The memory segment will provide bytes up to the given limit position.
-	 * 
-	 * @param segment The segment to read the next bytes from.
-	 * @param positionInSegment The position in the segment to start reading from.
-	 * @param limitInSegment The limit in the segment. When reached, the view will attempt to switch to
-	 *                       the next segment.
-	 */
-	protected void seekInput(MemorySegment segment, int positionInSegment, int limitInSegment) {
-		this.currentSegment = segment;
-		this.positionInSegment = positionInSegment;
-		this.limitInSegment = limitInSegment;
-	}
-	
-	/**
-	 * Clears the internal state of the view. After this call, all read attempts will fail, until the
-	 * {@link #advance()} or {@link #seekInput(MemorySegment, int, int)} method have been invoked.
-	 */
-	protected void clear() {
-		this.currentSegment = null;
-		this.positionInSegment = this.headerLength;
-		this.limitInSegment = headerLength;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                               Data Input Specific methods
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public int read(byte[] b) throws IOException{
-		return read(b,0,b.length);
-	}
-
-	@Override
-	public int read(byte[] b, int off, int len) throws IOException{
-		if (off < 0 || len < 0 || off + len > b.length) {
-			throw new IndexOutOfBoundsException();
-		}
-
-		int remaining = this.limitInSegment - this.positionInSegment;
-		if (remaining >= len) {
-			this.currentSegment.get(this.positionInSegment, b, off, len);
-			this.positionInSegment += len;
-			return len;
-		}
-		else {
-			if (remaining == 0) {
-				try {
-					advance();
-				}catch(EOFException eof){
-					return -1;
-				}
-				remaining = this.limitInSegment - this.positionInSegment;
-			}
-
-			int bytesRead = 0;
-			while (true) {
-				int toRead = Math.min(remaining, len-bytesRead);
-				this.currentSegment.get(this.positionInSegment, b, off, toRead);
-				off += toRead;
-				bytesRead += toRead;
-
-				if (len > bytesRead) {
-					try {
-						advance();
-					}catch(EOFException eof){
-						this.positionInSegment += toRead;
-						return bytesRead;
-					}
-					remaining = this.limitInSegment - this.positionInSegment;
-				}
-				else {
-					this.positionInSegment += toRead;
-					break;
-				}
-			}
-			return len;
-		}
-	}
-	
-	@Override
-	public void readFully(byte[] b) throws IOException {
-		readFully(b, 0, b.length);
-	}
-
-	@Override
-	public void readFully(byte[] b, int off, int len) throws IOException {
-		int bytesRead = read(b,off,len);
-
-		if(bytesRead < len){
-			throw new EOFException("There is no enough data left in the DataInputView.");
-		}
-	}
-
-	@Override
-	public boolean readBoolean() throws IOException {
-		return readByte() == 1;
-	}
-
-	@Override
-	public byte readByte() throws IOException {
-		if (this.positionInSegment < this.limitInSegment) {
-			return this.currentSegment.get(this.positionInSegment++);
-		}
-		else {
-			advance();
-			return readByte();
-		}
-	}
-
-	@Override
-	public int readUnsignedByte() throws IOException {
-		return readByte() & 0xff;
-	}
-
-	@Override
-	public short readShort() throws IOException {
-		if (this.positionInSegment < this.limitInSegment - 1) {
-			final short v = this.currentSegment.getShort(this.positionInSegment);
-			this.positionInSegment += 2;
-			return v;
-		}
-		else if (this.positionInSegment == this.limitInSegment) {
-			advance();
-			return readShort();
-		}
-		else {
-			return (short) ((readUnsignedByte() << 8) | readUnsignedByte());
-		}
-	}
-
-	@Override
-	public int readUnsignedShort() throws IOException {
-		if (this.positionInSegment < this.limitInSegment - 1) {
-			final int v = this.currentSegment.getShort(this.positionInSegment) & 0xffff;
-			this.positionInSegment += 2;
-			return v;
-		}
-		else if (this.positionInSegment == this.limitInSegment) {
-			advance();
-			return readUnsignedShort();
-		}
-		else {
-			return (readUnsignedByte() << 8) | readUnsignedByte();
-		}
-	}
-
-	@Override
-	public char readChar() throws IOException  {
-		if (this.positionInSegment < this.limitInSegment - 1) {
-			final char v = this.currentSegment.getChar(this.positionInSegment);
-			this.positionInSegment += 2;
-			return v;
-		}
-		else if (this.positionInSegment == this.limitInSegment) {
-			advance();
-			return readChar();
-		}
-		else {
-			return (char) ((readUnsignedByte() << 8) | readUnsignedByte());
-		}
-	}
-
-	@Override
-	public int readInt() throws IOException {
-		if (this.positionInSegment < this.limitInSegment - 3) {
-			final int v = this.currentSegment.getIntBigEndian(this.positionInSegment);
-			this.positionInSegment += 4;
-			return v;
-		}
-		else if (this.positionInSegment == this.limitInSegment) {
-			advance();
-			return readInt();
-		}
-		else {
-			return (readUnsignedByte() << 24) |
-				(readUnsignedByte() << 16) |
-				(readUnsignedByte() <<  8) |
-					readUnsignedByte();
-		}
-	}
-
-	@Override
-	public long readLong() throws IOException {
-		if (this.positionInSegment < this.limitInSegment - 7) {
-			final long v = this.currentSegment.getLongBigEndian(this.positionInSegment);
-			this.positionInSegment += 8;
-			return v;
-		}
-		else if (this.positionInSegment == this.limitInSegment) {
-			advance();
-			return readLong();
-		}
-		else {
-			long l = 0L;
-			l |= ((long) readUnsignedByte()) << 56;
-			l |= ((long) readUnsignedByte()) << 48;
-			l |= ((long) readUnsignedByte()) << 40;
-			l |= ((long) readUnsignedByte()) << 32;
-			l |= ((long) readUnsignedByte()) << 24;
-			l |= ((long) readUnsignedByte()) << 16;
-			l |= ((long) readUnsignedByte()) <<  8;
-			l |= (long) readUnsignedByte();
-			return l;
-		}
-	}
-
-	@Override
-	public float readFloat() throws IOException {
-		return Float.intBitsToFloat(readInt());
-	}
-
-	@Override
-	public double readDouble() throws IOException {
-		return Double.longBitsToDouble(readLong());
-	}
-
-	@Override
-	public String readLine() throws IOException {
-		final StringBuilder bld = new StringBuilder(32);
-		
-		try {
-			int b;
-			while ((b = readUnsignedByte()) != '\n') {
-				if (b != '\r') {
-					bld.append((char) b);
-				}
-			}
-		}
-		catch (EOFException eofex) {}
-
-		if (bld.length() == 0) {
-			return null;
-		}
-		
-		// trim a trailing carriage return
-		int len = bld.length();
-		if (len > 0 && bld.charAt(len - 1) == '\r') {
-			bld.setLength(len - 1);
-		}
-		return bld.toString();
-	}
-
-	@Override
-	public String readUTF() throws IOException {
-		final int utflen = readUnsignedShort();
-		
-		final byte[] bytearr;
-		final char[] chararr;
-		
-		if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
-			bytearr = new byte[utflen];
-			this.utfByteBuffer = bytearr;
-		} else {
-			bytearr = this.utfByteBuffer;
-		}
-		if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
-			chararr = new char[utflen];
-			this.utfCharBuffer = chararr;
-		} else {
-			chararr = this.utfCharBuffer;
-		}
-
-		int c, char2, char3;
-		int count = 0;
-		int chararr_count = 0;
-
-		readFully(bytearr, 0, utflen);
-
-		while (count < utflen) {
-			c = (int) bytearr[count] & 0xff;
-			if (c > 127) {
-				break;
-			}
-			count++;
-			chararr[chararr_count++] = (char) c;
-		}
-
-		while (count < utflen) {
-			c = (int) bytearr[count] & 0xff;
-			switch (c >> 4) {
-			case 0:
-			case 1:
-			case 2:
-			case 3:
-			case 4:
-			case 5:
-			case 6:
-			case 7:
-				/* 0xxxxxxx */
-				count++;
-				chararr[chararr_count++] = (char) c;
-				break;
-			case 12:
-			case 13:
-				/* 110x xxxx 10xx xxxx */
-				count += 2;
-				if (count > utflen) {
-					throw new UTFDataFormatException("malformed input: partial character at end");
-				}
-				char2 = (int) bytearr[count - 1];
-				if ((char2 & 0xC0) != 0x80) {
-					throw new UTFDataFormatException("malformed input around byte " + count);
-				}
-				chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
-				break;
-			case 14:
-				/* 1110 xxxx 10xx xxxx 10xx xxxx */
-				count += 3;
-				if (count > utflen) {
-					throw new UTFDataFormatException("malformed input: partial character at end");
-				}
-				char2 = (int) bytearr[count - 2];
-				char3 = (int) bytearr[count - 1];
-				if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
-					throw new UTFDataFormatException("malformed input around byte " + (count - 1));
-				}
-				chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
-				break;
-			default:
-				/* 10xx xxxx, 1111 xxxx */
-				throw new UTFDataFormatException("malformed input around byte " + count);
-			}
-		}
-		// The number of chars produced may be less than utflen
-		return new String(chararr, 0, chararr_count);
-	}
-	
-	@Override
-	public int skipBytes(int n) throws IOException {
-		if (n < 0) {
-			throw new IllegalArgumentException();
-		}
-		
-		int remaining = this.limitInSegment - this.positionInSegment;
-		if (remaining >= n) {
-			this.positionInSegment += n;
-			return n;
-		}
-		else {
-			if (remaining == 0) {
-				try {
-					advance();
-				} catch (EOFException eofex) {
-					return 0;
-				}
-				remaining = this.limitInSegment - this.positionInSegment;
-			}
-			
-			int skipped = 0;
-			while (true) {
-				int toSkip = Math.min(remaining, n);
-				n -= toSkip;
-				skipped += toSkip;
-				
-				if (n > 0) {
-					try {
-						advance();
-					} catch (EOFException eofex) {
-						return skipped;
-					}
-					remaining = this.limitInSegment - this.positionInSegment;	
-				}
-				else {
-					this.positionInSegment += toSkip;
-					break;
-				}
-			}
-			return skipped;
-		}
-	}
-
-	@Override
-	public void skipBytesToRead(int numBytes) throws IOException {
-		if (numBytes < 0) {
-			throw new IllegalArgumentException();
-		}
-		
-		int remaining = this.limitInSegment - this.positionInSegment;
-		if (remaining >= numBytes) {
-			this.positionInSegment += numBytes;
-		}
-		else {
-			if (remaining == 0) {
-				advance();
-				remaining = this.limitInSegment - this.positionInSegment;
-			}
-			
-			while (true) {
-				if (numBytes > remaining) {
-					numBytes -= remaining;
-					advance();
-					remaining = this.limitInSegment - this.positionInSegment;	
-				}
-				else {
-					this.positionInSegment += numBytes;
-					break;
-				}
-			}
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java
deleted file mode 100644
index ea73e62..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/AbstractPagedOutputView.java
+++ /dev/null
@@ -1,416 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import java.io.IOException;
-import java.io.UTFDataFormatException;
-
-import org.apache.flink.core.memory.DataInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.MemorySegment;
-
-
-/**
- * The base class for all output views that are backed by multiple memory pages. This base class contains all
- * encoding methods to write data to a page and detect page boundary crossing. The concrete sub classes must
- * implement the methods to collect the current page and provide the next memory page once the boundary is crossed.
- * <p>
- * The paging assumes that all memory segments are of the same size.
- */
-public abstract class AbstractPagedOutputView implements DataOutputView {
-	
-	private MemorySegment currentSegment;			// the current memory segment to write to
-	
-	protected final int segmentSize;				// the size of the memory segments
-	
-	protected final int headerLength;				// the number of bytes to skip at the beginning of each segment
-	
-	private int positionInSegment;					// the offset in the current segment
-	
-	private byte[] utfBuffer;						// the reusable array for UTF encodings
-	
-	
-	// --------------------------------------------------------------------------------------------
-	//                                    Constructors
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Creates a new output view that writes initially to the given initial segment. All segments in the
-	 * view have to be of the given {@code segmentSize}. A header of length {@code headerLength} is left
-	 * at the beginning of each segment.
-	 * 
-	 * @param initialSegment The segment that the view starts writing to.
-	 * @param segmentSize The size of the memory segments.
-	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
-	 */
-	protected AbstractPagedOutputView(MemorySegment initialSegment, int segmentSize, int headerLength) {
-		if (initialSegment == null) {
-			throw new NullPointerException("Initial Segment may not be null");
-		}
-		this.segmentSize = segmentSize;
-		this.headerLength = headerLength;
-		this.currentSegment = initialSegment;
-		this.positionInSegment = headerLength;
-	}
-	
-	/**
-	 * @param segmentSize The size of the memory segments.
-	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
-	 */
-	protected AbstractPagedOutputView(int segmentSize, int headerLength)
-	{
-		this.segmentSize = segmentSize;
-		this.headerLength = headerLength;
-	}
-	
-
-	// --------------------------------------------------------------------------------------------
-	//                                  Page Management
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * 
-	 * This method must return a segment. If no more segments are available, it must throw an
-	 * {@link java.io.EOFException}.
-	 * 
-	 * @param current The current memory segment
-	 * @param positionInCurrent The position in the segment, one after the last valid byte.
-	 * @return The next memory segment. 
-	 * 
-	 * @throws IOException
-	 */
-	protected abstract MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException;
-	
-	
-	/**
-	 * Gets the segment that the view currently writes to.
-	 * 
-	 * @return The segment the view currently writes to.
-	 */
-	public MemorySegment getCurrentSegment() {
-		return this.currentSegment;
-	}
-	
-	/**
-	 * Gets the current write position (the position where the next bytes will be written)
-	 * in the current memory segment.
-	 * 
-	 * @return The current write offset in the current memory segment.
-	 */
-	public int getCurrentPositionInSegment() {
-		return this.positionInSegment;
-	}
-	
-	/**
-	 * Gets the size of the segments used by this view.
-	 * 
-	 * @return The memory segment size.
-	 */
-	public int getSegmentSize() {
-		return this.segmentSize;
-	}
-	
-	/**
-	 * Moves the output view to the next page. This method invokes internally the
-	 * {@link #nextSegment(MemorySegment, int)} method to give the current memory segment to the concrete subclass' 
-	 * implementation and obtain the next segment to write to. Writing will continue inside the new segment
-	 * after the header.
-	 * 
-	 * @throws IOException Thrown, if the current segment could not be processed or a new segment could not
-	 *                     be obtained. 
-	 */
-	protected void advance() throws IOException {
-		this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
-		this.positionInSegment = this.headerLength;
-	}
-	
-	/**
-	 * Sets the internal state to the given memory segment and the given position within the segment. 
-	 * 
-	 * @param seg The memory segment to write the next bytes to.
-	 * @param position The position to start writing the next bytes to.
-	 */
-	protected void seekOutput(MemorySegment seg, int position) {
-		this.currentSegment = seg;
-		this.positionInSegment = position;
-	}
-	
-	/**
-	 * Clears the internal state. Any successive write calls will fail until either {@link #advance()} or
-	 * {@link #seekOutput(MemorySegment, int)} is called. 
-	 * 
-	 * @see #advance()
-	 * @see #seekOutput(MemorySegment, int)
-	 */
-	protected void clear() {
-		this.currentSegment = null;
-		this.positionInSegment = this.headerLength;
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	//                               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.putShort(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.putChar(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 >>> 0) & 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 >> 0) & 0x3F));
-			} else {
-				bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F));
-				bytearr[count++] = (byte) (0x80 | ((c >> 0) & 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;
-		}
-		return;
-	}
-
-	@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();
-		}
-		return;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java
deleted file mode 100644
index 85c91c0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/CheckedMemorySegment.java
+++ /dev/null
@@ -1,407 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-/**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- */
-public class CheckedMemorySegment {
-	
-	/**
-	 * The array in which the data is stored.
-	 */
-	protected byte[] memory;
-	
-	/**
-	 * The offset in the memory array where this segment starts.
-	 */
-	protected final int offset;
-	
-	/**
-	 * The size of the memory segment.
-	 */
-	protected final int size;
-	
-	/**
-	 * Wrapper for I/O requests.
-	 */
-	protected ByteBuffer wrapper;
-	
-	// -------------------------------------------------------------------------
-	//                             Constructors
-	// -------------------------------------------------------------------------
-
-	public CheckedMemorySegment(byte[] memory) {
-		this.memory = memory;
-		this.offset = 0;
-		this.size = memory.length;
-	}
-
-	// -------------------------------------------------------------------------
-	//                        MemorySegment Accessors
-	// -------------------------------------------------------------------------
-
-	public boolean isFreed() {
-		return this.memory == null;
-	}
-	
-	public final int size() {
-		return size;
-	}
-
-	public final byte[] getBackingArray() {
-		return this.memory;
-	}
-
-	public final int translateOffset(int offset) {
-		return this.offset + offset;
-	}
-	
-	// -------------------------------------------------------------------------
-	//                       Helper methods
-	// -------------------------------------------------------------------------
-
-	public ByteBuffer wrap(int offset, int length) {
-		if (offset > this.size || offset > this.size - length) {
-			throw new IndexOutOfBoundsException();
-		}
-		
-		if (this.wrapper == null) {
-			this.wrapper = ByteBuffer.wrap(this.memory, this.offset + offset, length);
-		}
-		else {
-			this.wrapper.position(this.offset + offset);
-			this.wrapper.limit(this.offset + offset + length);
-		}
-		
-		return this.wrapper;
-	}
-
-
-	// --------------------------------------------------------------------
-	//                            Random Access
-	// --------------------------------------------------------------------
-
-	// ------------------------------------------------------------------------------------------------------
-	// WARNING: Any code for range checking must take care to avoid integer overflows. The position
-	// integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
-	// <code>position + 3 &lt; end</code> may fail because <code>position + 3</code> becomes negative.
-	// A safe solution is to subtract the delta from the limit, for example
-	// <code>position &lt; end - 3</code>. Since all indices are always positive, and the integer domain
-	// has one more negative value than positive values, this can never cause an underflow.
-	// ------------------------------------------------------------------------------------------------------
-
-	public final byte get(int index) {
-		if (index >= 0 && index < this.size) {
-			return this.memory[this.offset + index];
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment put(int index, byte b) {
-		if (index >= 0 && index < this.size) {
-			this.memory[this.offset + index] = b;
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment get(int index, byte[] dst) {
-		return get(index, dst, 0, dst.length);
-	}
-
-	public final CheckedMemorySegment put(int index, byte[] src) {
-		return put(index, src, 0, src.length);
-	}
-
-	public final CheckedMemorySegment get(int index, byte[] dst, int offset, int length) {
-		if (index >= 0 && index < this.size && index <= this.size - length && offset <= dst.length - length) {
-			System.arraycopy(this.memory, this.offset + index, dst, offset, length);
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment put(int index, byte[] src, int offset, int length) {
-		if (index >= 0 && index < this.size && index <= this.size - length && offset <= src.length - length) {
-			System.arraycopy(src, offset, this.memory, this.offset + index, length);
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment get(DataOutput out, int offset, int length) throws IOException {
-		if (offset >= 0 && offset < this.size && length >= 0 && offset <= this.size - length) {
-			out.write(this.memory, this.offset + offset, length);
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment put(DataInput in, int offset, int length) throws IOException {
-		if (offset >= 0 && offset < this.size && length >= 0 && offset <= this.size - length) {
-			in.readFully(this.memory, this.offset + offset, length);
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final boolean getBoolean(int index) {
-		if (index >= 0 && index < this.size) {
-			return this.memory[this.offset + index] != 0;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment putBoolean(int index, boolean value) {
-		if (index >= 0 && index < this.size) {
-			this.memory[this.offset + index] = (byte) (value ? 1 : 0);
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final char getChar(int index) {
-		if (index >= 0 && index < this.size - 1) {
-			return (char) ( ((this.memory[this.offset + index + 0] & 0xff) << 8) | 
-							(this.memory[this.offset + index + 1] & 0xff) );
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment putChar(int index, char value) {
-		if (index >= 0 && index < this.size - 1) {
-			this.memory[this.offset + index + 0] = (byte) (value >> 8);
-			this.memory[this.offset + index + 1] = (byte) value;
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final short getShort(int index) {
-		if (index >= 0 && index < this.size - 1) {
-			return (short) (
-					((this.memory[this.offset + index + 0] & 0xff) << 8) |
-					((this.memory[this.offset + index + 1] & 0xff)) );
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final CheckedMemorySegment putShort(int index, short value) {
-		if (index >= 0 && index < this.size - 1) {
-			this.memory[this.offset + index + 0] = (byte) (value >> 8);
-			this.memory[this.offset + index + 1] = (byte) value;
-			return this;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final int getInt(int index) {
-		if (index >= 0 && index < this.size - 3) {
-			return ((this.memory[index    ] & 0xff) << 24)
-				| ((this.memory[index + 1] & 0xff) << 16)
-				| ((this.memory[index + 2] & 0xff) <<  8)
-				| ((this.memory[index + 3] & 0xff)     );
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final int getIntLittleEndian(int index) {
-		if (index >= 0 && index < this.size - 3) {
-			return ((this.memory[index    ] & 0xff)      )
-				| ((this.memory[index + 1] & 0xff) <<  8)
-				| ((this.memory[index + 2] & 0xff) << 16)
-				| ((this.memory[index + 3] & 0xff) << 24);
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final int getIntBigEndian(int index) {
-		if (index >= 0 && index < this.size - 3) {
-			return ((this.memory[index    ] & 0xff) << 24)
-				| ((this.memory[index + 1] & 0xff) << 16)
-				| ((this.memory[index + 2] & 0xff) <<  8)
-				| ((this.memory[index + 3] & 0xff)      );
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final void putInt(int index, int value) {
-		if (index >= 0 && index < this.size - 3) {
-			this.memory[index    ] = (byte) (value >> 24);
-			this.memory[index + 1] = (byte) (value >> 16);
-			this.memory[index + 2] = (byte) (value >> 8);
-			this.memory[index + 3] = (byte) value;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final void putIntLittleEndian(int index, int value) {
-		if (index >= 0 && index < this.size - 3) {
-			this.memory[index    ] = (byte) value;
-			this.memory[index + 1] = (byte) (value >>  8);
-			this.memory[index + 2] = (byte) (value >> 16);
-			this.memory[index + 3] = (byte) (value >> 24);
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final void putIntBigEndian(int index, int value) {
-		if (index >= 0 && index < this.size - 3) {
-			this.memory[index    ] = (byte) (value >> 24);
-			this.memory[index + 1] = (byte) (value >> 16);
-			this.memory[index + 2] = (byte) (value >> 8);
-			this.memory[index + 3] = (byte) value;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final long getLong(int index) {
-		if (index >= 0 && index < this.size - 7) {
-			return (((long) this.memory[index    ] & 0xff) << 56)
-				| (((long) this.memory[index + 1] & 0xff) << 48)
-				| (((long) this.memory[index + 2] & 0xff) << 40)
-				| (((long) this.memory[index + 3] & 0xff) << 32)
-				| (((long) this.memory[index + 4] & 0xff) << 24)
-				| (((long) this.memory[index + 5] & 0xff) << 16)
-				| (((long) this.memory[index + 6] & 0xff) <<  8)
-				| (((long) this.memory[index + 7] & 0xff)      );
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	public final long getLongLittleEndian(int index) {
-		if (index >= 0 && index < this.size - 7) {
-			return (((long) this.memory[index    ] & 0xff)      )
-				| (((long) this.memory[index + 1] & 0xff) <<  8)
-				| (((long) this.memory[index + 2] & 0xff) << 16)
-				| (((long) this.memory[index + 3] & 0xff) << 24)
-				| (((long) this.memory[index + 4] & 0xff) << 32)
-				| (((long) this.memory[index + 5] & 0xff) << 40)
-				| (((long) this.memory[index + 6] & 0xff) << 48)
-				| (((long) this.memory[index + 7] & 0xff) << 56);
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final long getLongBigEndian(int index) {
-		if (index >= 0 && index < this.size - 7) {
-			return (((long) this.memory[index    ] & 0xff) << 56)
-				| (((long) this.memory[index + 1] & 0xff) << 48)
-				| (((long) this.memory[index + 2] & 0xff) << 40)
-				| (((long) this.memory[index + 3] & 0xff) << 32)
-				| (((long) this.memory[index + 4] & 0xff) << 24)
-				| (((long) this.memory[index + 5] & 0xff) << 16)
-				| (((long) this.memory[index + 6] & 0xff) <<  8)
-				| (((long) this.memory[index + 7] & 0xff)      );
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final void putLong(int index, long value) {
-		if (index >= 0 && index < this.size - 7) {
-			this.memory[index    ] = (byte) (value >> 56);
-			this.memory[index + 1] = (byte) (value >> 48);
-			this.memory[index + 2] = (byte) (value >> 40);
-			this.memory[index + 3] = (byte) (value >> 32);
-			this.memory[index + 4] = (byte) (value >> 24);
-			this.memory[index + 5] = (byte) (value >> 16);
-			this.memory[index + 6] = (byte) (value >>  8);
-			this.memory[index + 7] = (byte)  value;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final void putLongLittleEndian(int index, long value) {
-		if (index >= 0 && index < this.size - 7) {
-			this.memory[index    ] = (byte)  value;
-			this.memory[index + 1] = (byte) (value >>  8);
-			this.memory[index + 2] = (byte) (value >> 16);
-			this.memory[index + 3] = (byte) (value >> 24);
-			this.memory[index + 4] = (byte) (value >> 32);
-			this.memory[index + 5] = (byte) (value >> 40);
-			this.memory[index + 6] = (byte) (value >> 48);
-			this.memory[index + 7] = (byte) (value >> 56);
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-	
-	public final void putLongBigEndian(int index, long value) {
-		if (index >= 0 && index < this.size - 7) {
-			this.memory[index    ] = (byte) (value >> 56);
-			this.memory[index + 1] = (byte) (value >> 48);
-			this.memory[index + 2] = (byte) (value >> 40);
-			this.memory[index + 3] = (byte) (value >> 32);
-			this.memory[index + 4] = (byte) (value >> 24);
-			this.memory[index + 5] = (byte) (value >> 16);
-			this.memory[index + 6] = (byte) (value >>  8);
-			this.memory[index + 7] = (byte)  value;
-		} else {
-			throw new IndexOutOfBoundsException();
-		}
-	}
-
-	public final float getFloat(int index) {
-		return Float.intBitsToFloat(getInt(index));
-	}
-
-	public final CheckedMemorySegment putFloat(int index, float value) {
-		putLong(index, Float.floatToIntBits(value));
-		return this;
-	}
-
-	public final double getDouble(int index) {
-		return Double.longBitsToDouble(getLong(index));
-	}
-
-	public final CheckedMemorySegment putDouble(int index, double value) {
-		putLong(index, Double.doubleToLongBits(value));
-		return this;
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java
deleted file mode 100644
index b041ac9..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/DefaultMemoryManager.java
+++ /dev/null
@@ -1,490 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.ConcurrentModificationException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-
-
-public class DefaultMemoryManager implements MemoryManager {
-	
-	/**
-	 * The default memory page size. Currently set to 32 KiBytes.
-	 */
-	public static final int DEFAULT_PAGE_SIZE = 32 * 1024;
-	
-	/**
-	 * The minimal memory page size. Currently set to 4 KiBytes.
-	 */
-	public static final int MIN_PAGE_SIZE = 4 * 1024;
-	
-	/**
-	 * The Logger.
-	 */
-	private static final Logger LOG = LoggerFactory.getLogger(DefaultMemoryManager.class);
-	
-	// --------------------------------------------------------------------------------------------
-	
-	private final Object lock = new Object();	 	// The lock used on the shared structures.
-	
-	private final ArrayDeque<byte[]> freeSegments;	// the free memory segments
-	
-	private final HashMap<AbstractInvokable, Set<DefaultMemorySegment>> allocatedSegments;
-	
-	private final long roundingMask;		// mask used to round down sizes to multiples of the page size
-	
-	private final int pageSize;				// the page size, in bytes
-	
-	private final int pageSizeBits;			// the number of bits that the power-of-two page size corresponds to
-	
-	private final int totalNumPages;		// The initial total size, for verification.
-
-	/** The total size of the memory managed by this memory manager */
-	private final long memorySize;
-
-	/** Number of slots of the task manager */
-	private final int numberOfSlots;
-	
-	private final boolean isPreAllocated;
-	
-	/** The number of memory pages that have not been allocated and are available for lazy allocation */
-	private int numNonAllocatedPages;
-	
-	/** flag whether the close() has already been invoked */
-	private boolean isShutDown;
-
-	// ------------------------------------------------------------------------
-	// Constructors / Destructors
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Creates a memory manager with the given capacity, using the default page size.
-	 * 
-	 * @param memorySize The total size of the memory to be managed by this memory manager.
-	 * @param numberOfSlots The number of slots of the task manager.
-	 */
-	public DefaultMemoryManager(long memorySize, int numberOfSlots) {
-		this(memorySize, numberOfSlots, DEFAULT_PAGE_SIZE, true);
-	}
-
-	/**
-	 * Creates a memory manager with the given capacity and given page size.
-	 * 
-	 * @param memorySize The total size of the memory to be managed by this memory manager.
-	 * @param numberOfSlots The number of slots of the task manager.
-	 * @param pageSize The size of the pages handed out by the memory manager.
-	 * @param preAllocateMemory True, if the memory manaber should immediately allocate all memory, false
-	 *                          if it should allocate and release the memory as needed.
-	 */
-	public DefaultMemoryManager(long memorySize, int numberOfSlots, int pageSize, boolean preAllocateMemory) {
-		// sanity checks
-		if (memorySize <= 0) {
-			throw new IllegalArgumentException("Size of total memory must be positive.");
-		}
-		if (pageSize < MIN_PAGE_SIZE) {
-			throw new IllegalArgumentException("The page size must be at least " + MIN_PAGE_SIZE + " bytes.");
-		}
-		if ((pageSize & (pageSize - 1)) != 0) {
-			// not a power of two
-			throw new IllegalArgumentException("The given page size is not a power of two.");
-		}
-
-		this.memorySize = memorySize;
-
-		this.numberOfSlots = numberOfSlots;
-		
-		// assign page size and bit utilities
-		this.pageSize = pageSize;
-		this.roundingMask = ~((long) (pageSize - 1));
-		int log = 0;
-		while ((pageSize = pageSize >>> 1) != 0) {
-			log++;
-		}
-		this.pageSizeBits = log;
-		
-		this.totalNumPages = getNumPages(memorySize);
-		if (this.totalNumPages < 1) {
-			throw new IllegalArgumentException("The given amount of memory amounted to less than one page.");
-		}
-		
-		// initialize the free segments and allocated segments tracking structures
-		this.freeSegments = new ArrayDeque<byte[]>(this.totalNumPages);
-		this.allocatedSegments = new HashMap<AbstractInvokable, Set<DefaultMemorySegment>>();
-
-		this.isPreAllocated = preAllocateMemory;
-		
-		if (preAllocateMemory) {
-			// add the full chunks
-			for (int i = 0; i < this.totalNumPages; i++) {
-				// allocate memory of the specified size
-				this.freeSegments.add(new byte[this.pageSize]);
-			}
-		}
-		else {
-			this.numNonAllocatedPages = this.totalNumPages;
-		}
-	}
-
-	@Override
-	public void shutdown() {
-		// -------------------- BEGIN CRITICAL SECTION -------------------
-		synchronized (this.lock)
-		{
-			if (!this.isShutDown) {
-				if (LOG.isDebugEnabled()) {
-					LOG.debug("Shutting down MemoryManager instance " + this);
-				}
-	
-				// mark as shutdown and release memory
-				this.isShutDown = true;
-				
-				this.freeSegments.clear();
-				this.numNonAllocatedPages = 0;
-				
-				// go over all allocated segments and release them
-				for (Set<DefaultMemorySegment> segments : this.allocatedSegments.values()) {
-					for (DefaultMemorySegment seg : segments) {
-						seg.destroy();
-					}
-				}
-			}
-		}
-		// -------------------- END CRITICAL SECTION -------------------
-	}
-
-	@Override
-	public boolean isShutdown() {
-		return this.isShutDown;
-	}
-
-	@Override
-	public boolean verifyEmpty() {
-		synchronized (this.lock) {
-			return isPreAllocated ?
-					this.freeSegments.size() == this.totalNumPages :
-					this.numNonAllocatedPages == this.totalNumPages;
-		}
-	}
-
-	// ------------------------------------------------------------------------
-	//                 MemoryManager interface implementation
-	// ------------------------------------------------------------------------
-
-	@Override
-	public List<MemorySegment> allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException {
-		final ArrayList<MemorySegment> segs = new ArrayList<MemorySegment>(numPages);
-		allocatePages(owner, segs, numPages);
-		return segs;
-	}
-
-	@Override
-	public void allocatePages(AbstractInvokable owner, List<MemorySegment> target, int numPages)
-			throws MemoryAllocationException
-	{
-		// sanity check
-		if (owner == null) {
-			throw new IllegalAccessError("The memory owner must not be null.");
-		}
-		
-		// reserve array space, if applicable
-		if (target instanceof ArrayList) {
-			((ArrayList<MemorySegment>) target).ensureCapacity(numPages);
-		}
-		
-		// -------------------- BEGIN CRITICAL SECTION -------------------
-		synchronized (this.lock)
-		{
-			if (this.isShutDown) {
-				throw new IllegalStateException("Memory manager has been shut down.");
-			}
-			
-			// in the case of pre-allocated memory, the 'numNonAllocatedPages' is zero, in the
-			// lazy case, the 'freeSegments.size()' is zero.
-			if (numPages > (this.freeSegments.size() + numNonAllocatedPages)) {
-				throw new MemoryAllocationException("Could not allocate " + numPages + " pages. Only " + 
-					this.freeSegments.size() + " pages are remaining.");
-			}
-			
-			Set<DefaultMemorySegment> segmentsForOwner = this.allocatedSegments.get(owner);
-			if (segmentsForOwner == null) {
-				segmentsForOwner = new HashSet<DefaultMemorySegment>(4 * numPages / 3 + 1);
-				this.allocatedSegments.put(owner, segmentsForOwner);
-			}
-			
-			if (isPreAllocated) {
-				for (int i = numPages; i > 0; i--) {
-					byte[] buffer = this.freeSegments.poll();
-					final DefaultMemorySegment segment = new DefaultMemorySegment(owner, buffer);
-					target.add(segment);
-					segmentsForOwner.add(segment);
-				}
-			}
-			else {
-				for (int i = numPages; i > 0; i--) {
-					byte[] buffer = new byte[pageSize];
-					final DefaultMemorySegment segment = new DefaultMemorySegment(owner, buffer);
-					target.add(segment);
-					segmentsForOwner.add(segment);
-				}
-				numNonAllocatedPages -= numPages;
-			}
-		}
-		// -------------------- END CRITICAL SECTION -------------------
-	}
-	
-	// ------------------------------------------------------------------------
-
-	@Override
-	public void release(MemorySegment segment) {
-		// check if segment is null or has already been freed
-		if (segment == null || segment.isFreed() || !(segment instanceof DefaultMemorySegment)) {
-			return;
-		}
-		
-		final DefaultMemorySegment defSeg = (DefaultMemorySegment) segment;
-		final AbstractInvokable owner = defSeg.owner;
-		
-		// -------------------- BEGIN CRITICAL SECTION -------------------
-		synchronized (this.lock)
-		{
-			if (this.isShutDown) {
-				throw new IllegalStateException("Memory manager has been shut down.");
-			}
-
-			// remove the reference in the map for the owner
-			try {
-				Set<DefaultMemorySegment> segsForOwner = this.allocatedSegments.get(owner);
-				
-				if (segsForOwner != null) {
-					segsForOwner.remove(defSeg);
-					if (segsForOwner.isEmpty()) {
-						this.allocatedSegments.remove(owner);
-					}
-				}
-
-				byte[] buffer = defSeg.destroy();
-				
-				if (isPreAllocated) {
-					// release the memory in any case
-					this.freeSegments.add(buffer);
-				}
-				else {
-					numNonAllocatedPages++;
-				}
-			}
-			catch (Throwable t) {
-				throw new RuntimeException("Error removing book-keeping reference to allocated memory segment.", t);
-			}
-		}
-		// -------------------- END CRITICAL SECTION -------------------
-	}
-
-	@Override
-	public <T extends MemorySegment> void release(Collection<T> segments) {
-		if (segments == null) {
-			return;
-		}
-		
-		// -------------------- BEGIN CRITICAL SECTION -------------------
-		synchronized (this.lock)
-		{
-			if (this.isShutDown) {
-				throw new IllegalStateException("Memory manager has been shut down.");
-			}
-
-			// since concurrent modifications to the collection
-			// can disturb the release, we need to try potentially multiple times
-			boolean successfullyReleased = false;
-			do {
-				final Iterator<T> segmentsIterator = segments.iterator();
-
-				AbstractInvokable lastOwner = null;
-				Set<DefaultMemorySegment> segsForOwner = null;
-
-				try {
-					// go over all segments
-					while (segmentsIterator.hasNext()) {
-
-						final MemorySegment seg = segmentsIterator.next();
-						if (seg == null || seg.isFreed()) {
-							continue;
-						}
-
-						final DefaultMemorySegment defSeg = (DefaultMemorySegment) seg;
-						final AbstractInvokable owner = defSeg.owner;
-
-						try {
-							// get the list of segments by this owner only if it is a different owner than for
-							// the previous one (or it is the first segment)
-							if (lastOwner != owner) {
-								lastOwner = owner;
-								segsForOwner = this.allocatedSegments.get(owner);
-							}
-
-							// remove the segment from the list
-							if (segsForOwner != null) {
-								segsForOwner.remove(defSeg);
-								if (segsForOwner.isEmpty()) {
-									this.allocatedSegments.remove(owner);
-								}
-							}
-
-							// release the memory in any case
-							byte[] buffer = defSeg.destroy();
-							
-							if (isPreAllocated) {
-								this.freeSegments.add(buffer);
-							}
-							else {
-								numNonAllocatedPages++;
-							}
-						}
-						catch (Throwable t) {
-							throw new RuntimeException(
-									"Error removing book-keeping reference to allocated memory segment.", t);
-						}
-					}
-
-					segments.clear();
-
-					// the only way to exit the loop
-					successfullyReleased = true;
-				}
-				catch (ConcurrentModificationException e) {
-					// this may happen in the case where an asynchronous
-					// call releases the memory. fall through the loop and try again
-				}
-			} while (!successfullyReleased);
-		}
-		// -------------------- END CRITICAL SECTION -------------------
-	}
-
-	@Override
-	public void releaseAll(AbstractInvokable owner) {
-		// -------------------- BEGIN CRITICAL SECTION -------------------
-		synchronized (this.lock)
-		{
-			if (this.isShutDown) {
-				throw new IllegalStateException("Memory manager has been shut down.");
-			}
-
-			// get all segments
-			final Set<DefaultMemorySegment> segments = this.allocatedSegments.remove(owner);
-
-			// all segments may have been freed previously individually
-			if (segments == null || segments.isEmpty()) {
-				return;
-			}
-
-			// free each segment
-			if (isPreAllocated) {
-				for (DefaultMemorySegment seg : segments) {
-					final byte[] buffer = seg.destroy();
-					this.freeSegments.add(buffer);
-				}
-			}
-			else {
-				for (DefaultMemorySegment seg : segments) {
-					seg.destroy();
-				}
-				numNonAllocatedPages += segments.size();
-			}
-
-			segments.clear();
-		}
-		// -------------------- END CRITICAL SECTION -------------------
-	}
-	
-	// ------------------------------------------------------------------------
-
-	@Override
-	public int getPageSize() {
-		return this.pageSize;
-	}
-
-	@Override
-	public long getMemorySize() {
-		return this.memorySize;
-	}
-
-	@Override
-	public int computeNumberOfPages(double fraction) {
-		if (fraction <= 0 || fraction > 1) {
-			throw new IllegalArgumentException("The fraction of memory to allocate must within (0, 1].");
-		}
-
-		return (int)(this.totalNumPages * fraction / this.numberOfSlots);
-	}
-
-	@Override
-	public long computeMemorySize(double fraction) {
-		return this.pageSize * computeNumberOfPages(fraction);
-	}
-
-	@Override
-	public long roundDownToPageSizeMultiple(long numBytes) {
-		return numBytes & this.roundingMask;
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private int getNumPages(long numBytes) {
-		if (numBytes < 0) {
-			throw new IllegalArgumentException("The number of bytes to allocate must not be negative.");
-		}
-		
-		final long numPages = numBytes >>> this.pageSizeBits;
-		if (numPages <= Integer.MAX_VALUE) {
-			return (int) numPages;
-		} else {
-			throw new IllegalArgumentException("The given number of bytes corresponds to more than MAX_INT pages.");
-		}
-	}
-	
-	// ------------------------------------------------------------------------
-	
-	private static final class DefaultMemorySegment extends MemorySegment {
-		
-		private AbstractInvokable owner;
-		
-		DefaultMemorySegment(AbstractInvokable owner, byte[] memory) {
-			super(memory);
-			this.owner = owner;
-		}
-		
-		byte[] destroy() {
-			final byte[] buffer = this.memory;
-			this.memory = null;
-			this.wrapper = null;
-			return buffer;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java
deleted file mode 100644
index cacb6b0..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/ListMemorySegmentSource.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import java.util.List;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.core.memory.MemorySegmentSource;
-
-/**
- * Simple memory segment source that draws segments from a list.
- * 
- */
-public class ListMemorySegmentSource implements MemorySegmentSource
-{
-	private final List<MemorySegment> segments;
-	
-	public ListMemorySegmentSource(final List<MemorySegment> memorySegments) {
-		this.segments = memorySegments;
-	}
-	
-
-	@Override
-	public MemorySegment nextSegment() {
-		if (this.segments.size() > 0) {
-			return this.segments.remove(this.segments.size() - 1);
-		} else {
-			return null;
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java
deleted file mode 100644
index d4d8b3b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryAllocationException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-/**
- * An exception to be thrown when a memory allocation operation is not successful.
- */
-public class MemoryAllocationException extends Exception {
-	
-	private static final long serialVersionUID = -403983866457947012L;
-
-	public MemoryAllocationException() {
-		super();
-	}
-
-	public MemoryAllocationException(String message, Throwable cause) {
-		super(message, cause);
-	}
-
-	public MemoryAllocationException(String message) {
-		super(message);
-	}
-
-	public MemoryAllocationException(Throwable cause) {
-		super(cause);
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java
deleted file mode 100644
index 631f0b8..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/MemoryManager.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-
-/**
- * Interface for a memory manager that assigns portions of memory to different tasks. Each allocated segment of
- * memory is specific to a task. Memory segments can be freed individually, or all memory allocated by a task can 
- * be freed as a whole.
- * <p>
- * Internally, memory is represented as byte arrays. The memory manager acts like a distributer for memory, which
- * means it assigns portions of the arrays to tasks. If memory is released, it means that this part of the memory can
- * be assigned to other tasks.
- */
-public interface MemoryManager {
-	
-	List<MemorySegment> allocatePages(AbstractInvokable owner, int numPages) throws MemoryAllocationException;
-	
-	void allocatePages(AbstractInvokable owner, List<MemorySegment> target, int numPages) throws MemoryAllocationException;
-	
-	/**
-	 * Tries to release the memory for the specified segment. If the <code>segment</code> has already been released or
-	 * is <code>null</code>, the request is simply ignored. If the segment is not from the expected
-	 * MemorySegment implementation type, an <code>IllegalArgumentException</code> is thrown.
-	 * 
-	 * @param segment The segment to be released.
-	 * @throws IllegalArgumentException Thrown, if the given segment is of an incompatible type.
-	 */
-	void release(MemorySegment segment);
-
-	/**
-	 * Tries to release the memory for the specified collection of segments.
-	 * 
-	 * @param <T> The type of memory segment.
-	 * @param segments The segments to be released.
-	 * @throws NullPointerException Thrown, if the given collection is null.
-	 * @throws IllegalArgumentException Thrown, id the segments are of an incompatible type.
-	 */
-	<T extends MemorySegment> void release(Collection<T> segments);
-	
-	/**
-	 * Releases all memory segments for the given task. 
-	 *
-	 * @param task The task whose memory segments are to be released.
-	 */
-	void releaseAll(AbstractInvokable task);
-	
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Gets the size of the pages handled by the memory manager.
-	 * 
-	 * @return The size of the pages handled by the memory manager.
-	 */
-	int getPageSize();
-
-	/**
-	 * Returns the total size of memory.
-	 * 
-	 * @return The total size of memory.
-	 */
-	long getMemorySize();
-	
-	/**
-	 * Computes to how many pages the given number of bytes corresponds. If the given number of bytes is not an
-	 * exact multiple of a page size, the result is rounded down, such that a portion of the memory (smaller
-	 * than the page size) is not included.
-	 * 
-	 * @param fraction the fraction of the total memory per slot
-	 * @return The number of pages to which 
-	 */
-	int computeNumberOfPages(double fraction);
-
-	/**
-	 * Computes the memory size of the fraction per slot.
-	 * 
-	 * @param fraction The fraction of the memory of the task slot.
-	 * @return The number of pages corresponding to the memory fraction.
-	 */
-	long computeMemorySize(double fraction);
-	
-	/**
-	 * Rounds the given value down to a multiple of the memory manager's page size.
-	 * 
-	 * @return The given value, rounded down to a multiple of the page size.
-	 */
-	long roundDownToPageSizeMultiple(long numBytes);
-
-	// --------------------------------------------------------------------------------------------
-	
-	/**
-	 * Shuts the memory manager down, trying to release all the memory it managed. Depending
-	 * on implementation details, the memory does not necessarily become reclaimable by the
-	 * garbage collector, because there might still be references to allocated segments in the
-	 * code that allocated them from the memory manager.
-	 */
-	void shutdown();
-
-	/**
-	 * Checks whether the MemoryManager has been shut down.
-	 *
-	 * @return True, if the memory manager is shut down, false otherwise.
-	 */
-	boolean isShutdown();
-
-	/**
-	 * Checks if the memory manager all memory available.
-	 * 
-	 * @return True, if the memory manager is empty and valid, false if it is not empty or corrupted.
-	 */
-	boolean verifyEmpty();
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java
deleted file mode 100644
index 7219503..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/SimpleMemorySegment.java
+++ /dev/null
@@ -1,329 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-/**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- */
-public class SimpleMemorySegment {
-	
-	/**
-	 * The array in which the data is stored.
-	 */
-	protected byte[] memory;
-	
-	/**
-	 * Wrapper for I/O requests.
-	 */
-	protected ByteBuffer wrapper;
-	
-	// -------------------------------------------------------------------------
-	//                             Constructors
-	// -------------------------------------------------------------------------
-
-	/**
-	 * Creates a new memory segment of given size with the provided views.
-	 */
-	public SimpleMemorySegment(byte[] memory) {
-		this.memory = memory;
-	}
-
-	// -------------------------------------------------------------------------
-	//                        MemorySegment Accessors
-	// -------------------------------------------------------------------------
-	
-	/**
-	 * Checks whether this memory segment has already been freed. In that case, the
-	 * segment must not be used any more.
-	 * 
-	 * @return True, if the segment has been freed, false otherwise.
-	 */
-	public final boolean isFreed() {
-		return this.memory == null;
-	}
-	
-	/**
-	 * Gets the size of the memory segment, in bytes. Because segments
-	 * are backed by arrays, they cannot be larger than two GiBytes.
-	 * 
-	 * @return The size in bytes.
-	 */
-	public final int size() {
-		return this.memory.length;
-	}
-	
-	/**
-	 * Gets the byte array that backs the memory segment and this random access view.
-	 * Since different regions of the backing array are used by different segments, the logical
-	 * positions in this view do not correspond to the indexes in the backing array and need
-	 * to be translated via the {@link #translateOffset(int)} method.
-	 * 
-	 * @return The backing byte array.
-	 */
-	public final byte[] getBackingArray() {
-		return this.memory;
-	}
-
-	/**
-	 * Translates the given offset for this view into the offset for the backing array.
-	 * 
-	 * @param offset The offset to be translated.
-	 * @return The corresponding position in the backing array.
-	 */
-	public final int translateOffset(int offset) {
-		return offset;
-	}
-	
-	// -------------------------------------------------------------------------
-	//                       Helper methods
-	// -------------------------------------------------------------------------
-	
-
-	/**
-	 * Wraps the chunk of the underlying memory located between <tt>offset<tt> and 
-	 * <tt>length</tt> in a NIO ByteBuffer.
-	 * 
-	 * @param offset The offset in the memory segment.
-	 * @param length The number of bytes to be wrapped as a buffer.
-	 * @return A <tt>ByteBuffer</tt> backed by the specified portion of the memory segment.
-	 * @throws IndexOutOfBoundsException Thrown, if offset is negative or larger than the memory segment size,
-	 *                                   or if the offset plus the length is larger than the segment size.
-	 */
-	public ByteBuffer wrap(int offset, int length) {
-		if (offset > this.memory.length || offset > this.memory.length - length) {
-			throw new IndexOutOfBoundsException();
-		}
-		
-		if (this.wrapper == null) {
-			this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
-		}
-		else {
-			this.wrapper.position(offset);
-			this.wrapper.limit(offset + length);
-		}
-		
-		return this.wrapper;
-	}
-
-
-	// --------------------------------------------------------------------
-	//                            Random Access
-	// --------------------------------------------------------------------
-
-	// ------------------------------------------------------------------------------------------------------
-	// WARNING: Any code for range checking must take care to avoid integer overflows. The position
-	// integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
-	// <code>position + 3 &lt; end</code> may fail because <code>position + 3</code> becomes negative.
-	// A safe solution is to subtract the delta from the limit, for example
-	// <code>position &lt; end - 3</code>. Since all indices are always positive, and the integer domain
-	// has one more negative value than positive values, this can never cause an underflow.
-	// ------------------------------------------------------------------------------------------------------
-
-	public final byte get(int index) {
-		return this.memory[index];
-	}
-
-	public final void put(int index, byte b) {
-		this.memory[index] = b;
-	}
-
-	public final void get(int index, byte[] dst) {
-		get(index, dst, 0, dst.length);
-	}
-
-	public final void put(int index, byte[] src) {
-		put(index, src, 0, src.length);
-	}
-
-	public final void get(int index, byte[] dst, int offset, int length) {
-		System.arraycopy(this.memory, index, dst, offset, length);
-	}
-
-	public final void put(int index, byte[] src, int offset, int length) {
-		System.arraycopy(src, offset, this.memory, index, length);
-	}
-
-	public final void get(DataOutput out, int offset, int length) throws IOException {
-		out.write(this.memory, offset, length);
-	}
-
-	public final void put(DataInput in, int offset, int length) throws IOException {
-		in.readFully(this.memory, offset, length);
-	}
-
-	public final boolean getBoolean(int index) {
-		return this.memory[index] != 0;
-	}
-
-	public final void putBoolean(int index, boolean value) {
-		this.memory[index] = (byte) (value ? 1 : 0);
-	}
-
-	public final char getChar(int index) {
-		return (char) ( ((this.memory[index    ] & 0xff) << 8) | 
-						(this.memory[index + 1] & 0xff) );
-	}
-
-	public final void putChar(int index, char value) {
-		this.memory[index    ] = (byte) (value >> 8);
-		this.memory[index + 1] = (byte) value;
-	}
-
-	public final short getShort(int index) {
-		return (short) (
-				((this.memory[index    ] & 0xff) << 8) |
-				((this.memory[index + 1] & 0xff)) );
-	}
-
-	public final void putShort(int index, short value) {
-		this.memory[index    ] = (byte) (value >> 8);
-		this.memory[index + 1] = (byte) value;
-	}
-	
-	public final int getInt(int index) {
-		return ((this.memory[index    ] & 0xff) << 24)
-			| ((this.memory[index + 1] & 0xff) << 16)
-			| ((this.memory[index + 2] & 0xff) <<  8)
-			| ((this.memory[index + 3] & 0xff)     );
-	}
-
-	public final int getIntLittleEndian(int index) {
-		return ((this.memory[index    ] & 0xff)      )
-			| ((this.memory[index + 1] & 0xff) <<  8)
-			| ((this.memory[index + 2] & 0xff) << 16)
-			| ((this.memory[index + 3] & 0xff) << 24);
-	}
-	
-	public final int getIntBigEndian(int index) {
-		return ((this.memory[index    ] & 0xff) << 24)
-			| ((this.memory[index + 1] & 0xff) << 16)
-			| ((this.memory[index + 2] & 0xff) <<  8)
-			| ((this.memory[index + 3] & 0xff)      );
-	}
-	
-	public final void putInt(int index, int value) {
-		this.memory[index    ] = (byte) (value >> 24);
-		this.memory[index + 1] = (byte) (value >> 16);
-		this.memory[index + 2] = (byte) (value >> 8);
-		this.memory[index + 3] = (byte) value;
-	}
-	
-	public final void putIntLittleEndian(int index, int value) {
-		this.memory[index    ] = (byte) value;
-		this.memory[index + 1] = (byte) (value >>  8);
-		this.memory[index + 2] = (byte) (value >> 16);
-		this.memory[index + 3] = (byte) (value >> 24);
-	}
-	
-	public final void putIntBigEndian(int index, int value) {
-		this.memory[index    ] = (byte) (value >> 24);
-		this.memory[index + 1] = (byte) (value >> 16);
-		this.memory[index + 2] = (byte) (value >> 8);
-		this.memory[index + 3] = (byte) value;
-	}
-	
-	public final long getLong(int index) {
-		return (((long) this.memory[index    ] & 0xff) << 56)
-			| (((long) this.memory[index + 1] & 0xff) << 48)
-			| (((long) this.memory[index + 2] & 0xff) << 40)
-			| (((long) this.memory[index + 3] & 0xff) << 32)
-			| (((long) this.memory[index + 4] & 0xff) << 24)
-			| (((long) this.memory[index + 5] & 0xff) << 16)
-			| (((long) this.memory[index + 6] & 0xff) <<  8)
-			| (((long) this.memory[index + 7] & 0xff)      );
-	}
-	public final long getLongLittleEndian(int index) {
-		return (((long) this.memory[index    ] & 0xff)      )
-			| (((long) this.memory[index + 1] & 0xff) <<  8)
-			| (((long) this.memory[index + 2] & 0xff) << 16)
-			| (((long) this.memory[index + 3] & 0xff) << 24)
-			| (((long) this.memory[index + 4] & 0xff) << 32)
-			| (((long) this.memory[index + 5] & 0xff) << 40)
-			| (((long) this.memory[index + 6] & 0xff) << 48)
-			| (((long) this.memory[index + 7] & 0xff) << 56);
-	}
-	
-	public final long getLongBigEndian(int index) {
-		return (((long) this.memory[index    ] & 0xff) << 56)
-			| (((long) this.memory[index + 1] & 0xff) << 48)
-			| (((long) this.memory[index + 2] & 0xff) << 40)
-			| (((long) this.memory[index + 3] & 0xff) << 32)
-			| (((long) this.memory[index + 4] & 0xff) << 24)
-			| (((long) this.memory[index + 5] & 0xff) << 16)
-			| (((long) this.memory[index + 6] & 0xff) <<  8)
-			| (((long) this.memory[index + 7] & 0xff)      );
-	}
-
-	public final void putLong(int index, long value) {
-		this.memory[index    ] = (byte) (value >> 56);
-		this.memory[index + 1] = (byte) (value >> 48);
-		this.memory[index + 2] = (byte) (value >> 40);
-		this.memory[index + 3] = (byte) (value >> 32);
-		this.memory[index + 4] = (byte) (value >> 24);
-		this.memory[index + 5] = (byte) (value >> 16);
-		this.memory[index + 6] = (byte) (value >>  8);
-		this.memory[index + 7] = (byte)  value;
-	}
-	
-	public final void putLongLittleEndian(int index, long value) {
-		this.memory[index    ] = (byte)  value;
-		this.memory[index + 1] = (byte) (value >>  8);
-		this.memory[index + 2] = (byte) (value >> 16);
-		this.memory[index + 3] = (byte) (value >> 24);
-		this.memory[index + 4] = (byte) (value >> 32);
-		this.memory[index + 5] = (byte) (value >> 40);
-		this.memory[index + 6] = (byte) (value >> 48);
-		this.memory[index + 7] = (byte) (value >> 56);
-	}
-	
-	public final void putLongBigEndian(int index, long value) {
-		this.memory[index    ] = (byte) (value >> 56);
-		this.memory[index + 1] = (byte) (value >> 48);
-		this.memory[index + 2] = (byte) (value >> 40);
-		this.memory[index + 3] = (byte) (value >> 32);
-		this.memory[index + 4] = (byte) (value >> 24);
-		this.memory[index + 5] = (byte) (value >> 16);
-		this.memory[index + 6] = (byte) (value >>  8);
-		this.memory[index + 7] = (byte)  value;
-	}
-	
-	public final float getFloat(int index) {
-		return Float.intBitsToFloat(getInt(index));
-	}
-
-	public final void putFloat(int index, float value) {
-		putLong(index, Float.floatToIntBits(value));
-	}
-	
-	public final double getDouble(int index) {
-		return Double.longBitsToDouble(getLong(index));
-	}
-
-	public final void putDouble(int index, double value) {
-		putLong(index, Double.doubleToLongBits(value));
-	}
-}


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

Posted by se...@apache.org.
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);
+		}
+	}
+}


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java
deleted file mode 100644
index 21d3e27..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/memorymanager/UnsafeMemorySegment.java
+++ /dev/null
@@ -1,391 +0,0 @@
-/*
- * 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.runtime.memorymanager;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-
-import org.apache.flink.core.memory.MemoryUtils;
-
-/**
- * This class represents a piece of memory allocated from the memory manager. The segment is backed
- * by a byte array and features random put and get methods for the basic types that are stored in a byte-wise
- * fashion in the memory.
- */
-public class UnsafeMemorySegment {
-	
-	// flag to enable / disable boundary checks. Note that the compiler eliminates the check code
-	// paths (as dead code) when this constant is set to false.
-	private static final boolean CHECKED = false;
-	
-	/**
-	 * The array in which the data is stored.
-	 */
-	protected byte[] memory;
-	
-	/**
-	 * Wrapper for I/O requests.
-	 */
-	protected ByteBuffer wrapper;
-	
-	// -------------------------------------------------------------------------
-	//                             Constructors
-	// -------------------------------------------------------------------------
-
-	/**
-	 * Creates a new memory segment of given size with the provided views.
-	 */
-	public UnsafeMemorySegment(byte[] memory) {
-		this.memory = memory;
-	}
-
-	// -------------------------------------------------------------------------
-	//                        MemorySegment Accessors
-	// -------------------------------------------------------------------------
-	
-	/**
-	 * Checks whether this memory segment has already been freed. In that case, the
-	 * segment must not be used any more.
-	 * 
-	 * @return True, if the segment has been freed, false otherwise.
-	 */
-	public final boolean isFreed() {
-		return this.memory == null;
-	}
-	
-	/**
-	 * Gets the size of the memory segment, in bytes. Because segments
-	 * are backed by arrays, they cannot be larger than two GiBytes.
-	 * 
-	 * @return The size in bytes.
-	 */
-	public final int size() {
-		return this.memory.length;
-	}
-	
-	/**
-	 * Gets the byte array that backs the memory segment and this random access view.
-	 * Since different regions of the backing array are used by different segments, the logical
-	 * positions in this view do not correspond to the indexes in the backing array and need
-	 * to be translated via the {@link #translateOffset(int)} method.
-	 * 
-	 * @return The backing byte array.
-	 */
-	@Deprecated
-	public final byte[] getBackingArray() {
-		return this.memory;
-	}
-
-	/**
-	 * Translates the given offset for this view into the offset for the backing array.
-	 * 
-	 * @param offset The offset to be translated.
-	 * @return The corresponding position in the backing array.
-	 */
-	@Deprecated
-	public final int translateOffset(int offset) {
-		return offset;
-	}
-	
-	// -------------------------------------------------------------------------
-	//                       Helper methods
-	// -------------------------------------------------------------------------
-	
-
-	/**
-	 * Wraps the chunk of the underlying memory located between <tt>offset<tt> and 
-	 * <tt>length</tt> in a NIO ByteBuffer.
-	 * 
-	 * @param offset The offset in the memory segment.
-	 * @param length The number of bytes to be wrapped as a buffer.
-	 * @return A <tt>ByteBuffer</tt> backed by the specified portion of the memory segment.
-	 * @throws IndexOutOfBoundsException Thrown, if offset is negative or larger than the memory segment size,
-	 *                                   or if the offset plus the length is larger than the segment size.
-	 */
-	public ByteBuffer wrap(int offset, int length) {
-		if (offset > this.memory.length || offset > this.memory.length - length) {
-			throw new IndexOutOfBoundsException();
-		}
-		
-		if (this.wrapper == null) {
-			this.wrapper = ByteBuffer.wrap(this.memory, offset, length);
-		}
-		else {
-			this.wrapper.position(offset);
-			this.wrapper.limit(offset + length);
-		}
-		
-		return this.wrapper;
-	}
-
-
-	// --------------------------------------------------------------------
-	//                            Random Access
-	// --------------------------------------------------------------------
-
-	// ------------------------------------------------------------------------------------------------------
-	// WARNING: Any code for range checking must take care to avoid integer overflows. The position
-	// integer may go up to <code>Integer.MAX_VALUE</tt>. Range checks that work after the principle
-	// <code>position + 3 &lt; end</code> may fail because <code>position + 3</code> becomes negative.
-	// A safe solution is to subtract the delta from the limit, for example
-	// <code>position &lt; end - 3</code>. Since all indices are always positive, and the integer domain
-	// has one more negative value than positive values, this can never cause an underflow.
-	// ------------------------------------------------------------------------------------------------------
-
-	public final byte get(int index) {
-		return this.memory[index];
-	}
-
-	public final void put(int index, byte b) {
-		this.memory[index] = b;
-	}
-
-	public final void get(int index, byte[] dst) {
-		get(index, dst, 0, dst.length);
-	}
-
-	public final void put(int index, byte[] src) {
-		put(index, src, 0, src.length);
-	}
-
-	public final void get(int index, byte[] dst, int offset, int length) {
-		System.arraycopy(this.memory, index, dst, offset, length);
-	}
-
-	public final void put(int index, byte[] src, int offset, int length) {
-		System.arraycopy(src, offset, this.memory, index, length);
-	}
-
-	public final void get(DataOutput out, int offset, int length) throws IOException {
-		out.write(this.memory, offset, length);
-	}
-
-	public final void put(DataInput in, int offset, int length) throws IOException {
-		in.readFully(this.memory, offset, length);
-	}
-
-	public final boolean getBoolean(int index) {
-		return this.memory[index] != 0;
-	}
-
-	public final void putBoolean(int index, boolean value) {
-		this.memory[index] = (byte) (value ? 1 : 0);
-	}
-
-	public final char getChar(int index) {
-		return (char) ( ((this.memory[index    ] & 0xff) << 8) | 
-						(this.memory[index + 1] & 0xff) );
-	}
-
-	public final void putChar(int index, char value) {
-		this.memory[index    ] = (byte) (value >> 8);
-		this.memory[index + 1] = (byte) value;
-	}
-
-	public final short getShort(int index) {
-		return (short) (
-				((this.memory[index    ] & 0xff) << 8) |
-				((this.memory[index + 1] & 0xff)) );
-	}
-
-	public final void putShort(int index, short value) {
-		this.memory[index    ] = (byte) (value >> 8);
-		this.memory[index + 1] = (byte) value;
-	}
-
-	@SuppressWarnings("restriction")
-	public final int getInt(int index) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 4) {
-				return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			return UNSAFE.getInt(this.memory, BASE_OFFSET + index);
-		}
-	}
-	
-	public final int getIntLittleEndian(int index) {
-		if (LITTLE_ENDIAN) {
-			return getInt(index);
-		} else {
-			return Integer.reverseBytes(getInt(index));
-		}
-	}
-	
-	public final int getIntBigEndian(int index) {
-		if (LITTLE_ENDIAN) {
-			return Integer.reverseBytes(getInt(index));
-		} else {
-			return getInt(index);
-		}
-	}
-
-	@SuppressWarnings("restriction")
-	public final void putInt(int index, int value) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 4) {
-				UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			UNSAFE.putInt(this.memory, BASE_OFFSET + index, value);
-		}
-	}
-	
-	public final void putIntLittleEndian(int index, int value) {
-		if (LITTLE_ENDIAN) {
-			putInt(index, value);
-		} else {
-			putInt(index, Integer.reverseBytes(value));
-		}
-	}
-	
-	public final void putIntBigEndian(int index, int value) {
-		if (LITTLE_ENDIAN) {
-			putInt(index, Integer.reverseBytes(value));
-		} else {
-			putInt(index, value);
-		}
-	}
-
-	@SuppressWarnings("restriction")
-	public final long getLong(int index) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 8) {
-				return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			return UNSAFE.getLong(this.memory, BASE_OFFSET + index);
-		}
-	}
-	
-	public final long getLongLittleEndian(int index) {
-		if (LITTLE_ENDIAN) {
-			return getLong(index);
-		} else {
-			return Long.reverseBytes(getLong(index));
-		}
-	}
-	
-	public final long getLongBigEndian(int index) {
-		if (LITTLE_ENDIAN) {
-			return Long.reverseBytes(getLong(index));
-		} else {
-			return getLong(index);
-		}
-	}
-
-	@SuppressWarnings("restriction")
-	public final void putLong(int index, long value) {
-		if (CHECKED) {
-			if (index >= 0 && index <= this.memory.length - 8) {
-				UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
-			} else {
-				throw new IndexOutOfBoundsException();
-			}
-		} else {
-			UNSAFE.putLong(this.memory, BASE_OFFSET + index, value);
-		}
-	}
-
-	public final void putLongLittleEndian(int index, long value) {
-		if (LITTLE_ENDIAN) {
-			putLong(index, value);
-		} else {
-			putLong(index, Long.reverseBytes(value));
-		}
-	}
-	
-	public final void putLongBigEndian(int index, long value) {
-		if (LITTLE_ENDIAN) {
-			putLong(index, Long.reverseBytes(value));
-		} else {
-			putLong(index, value);
-		}
-	}
-	
-	public final float getFloat(int index) {
-		return Float.intBitsToFloat(getInt(index));
-	}
-	
-	public final float getFloatLittleEndian(int index) {
-		return Float.intBitsToFloat(getIntLittleEndian(index));
-	}
-	
-	public final float getFloatBigEndian(int index) {
-		return Float.intBitsToFloat(getIntBigEndian(index));
-	}
-
-	public final void putFloat(int index, float value) {
-		putInt(index, Float.floatToRawIntBits(value));
-	}
-	
-	public final void putFloatLittleEndian(int index, float value) {
-		putIntLittleEndian(index, Float.floatToRawIntBits(value));
-	}
-	
-	public final void putFloatBigEndian(int index, float value) {
-		putIntBigEndian(index, Float.floatToRawIntBits(value));
-	}
-	
-	public final double getDouble(int index) {
-		return Double.longBitsToDouble(getLong(index));
-	}
-	
-	public final double getDoubleLittleEndian(int index) {
-		return Double.longBitsToDouble(getLongLittleEndian(index));
-	}
-	
-	public final double getDoubleBigEndian(int index) {
-		return Double.longBitsToDouble(getLongBigEndian(index));
-	}
-
-	public final void putDouble(int index, double value) {
-		putLong(index, Double.doubleToRawLongBits(value));
-	}
-
-	public final void putDoubleLittleEndian(int index, double value) {
-		putLongLittleEndian(index, Double.doubleToRawLongBits(value));
-	}
-
-	public final void putDoubleBigEndian(int index, double value) {
-		putLongBigEndian(index, Double.doubleToRawLongBits(value));
-	}
-	
-	// --------------------------------------------------------------------------------------------
-	// Utilities for native memory accesses and checks
-	// --------------------------------------------------------------------------------------------
-	
-	@SuppressWarnings("restriction")
-	private static final sun.misc.Unsafe UNSAFE = MemoryUtils.UNSAFE;
-	
-	@SuppressWarnings("restriction")
-	private static final long BASE_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
-	
-	private static final boolean LITTLE_ENDIAN = (MemoryUtils.NATIVE_BYTE_ORDER == ByteOrder.LITTLE_ENDIAN);
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
index 89b7709..38c74e0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/AbstractOuterJoinDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.util.Collector;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
index cc39341..011c980 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/CrossDriver.java
@@ -24,7 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.api.common.functions.CrossFunction;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.resettable.BlockResettableMutableObjectIterator;
 import org.apache.flink.runtime.operators.resettable.SpillingResettableMutableObjectIterator;
 import org.apache.flink.runtime.operators.util.TaskConfig;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
index 998064a..30786aa 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/FullOuterJoinDriver.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
index 2bf778e..028ed95 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/GroupReduceCombineDriver.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
 import org.apache.flink.runtime.operators.sort.InMemorySorter;
 import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
index 95e98ce..811f00c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/JoinDriver.java
@@ -25,7 +25,7 @@ import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.hash.NonReusingBuildFirstHashMatchIterator;
 import org.apache.flink.runtime.operators.hash.NonReusingBuildSecondHashMatchIterator;
 import org.apache.flink.runtime.operators.hash.ReusingBuildFirstHashMatchIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
index b8932af..3cccab8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/LeftOuterJoinDriver.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
index 5c2ed67..baeda3a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/PactTaskContext.java
@@ -24,7 +24,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.util.TaskConfig;
 import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo;
 import org.apache.flink.util.Collector;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
index f3ea700..19557bc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/ReduceCombineDriver.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
 import org.apache.flink.runtime.operators.sort.InMemorySorter;
 import org.apache.flink.runtime.operators.sort.NormalizedKeySorter;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
index 3cefbba..6d35f92 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RegularPactTask.java
@@ -42,7 +42,7 @@ import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.chaining.ChainedDriver;
 import org.apache.flink.runtime.operators.chaining.ExceptionInChainedStubException;
 import org.apache.flink.runtime.operators.resettable.SpillingResettableMutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
index c455dc4..c93637e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/RightOuterJoinDriver.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparatorFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.sort.AbstractMergeOuterJoinIterator.OuterJoinType;
 import org.apache.flink.runtime.operators.sort.NonReusingMergeOuterJoinIterator;
 import org.apache.flink.runtime.operators.sort.ReusingMergeOuterJoinIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
index 63f2fd5..59a0f59 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/TempBarrier.java
@@ -30,9 +30,9 @@ import org.apache.flink.runtime.io.disk.InputViewIterator;
 import org.apache.flink.runtime.io.disk.SpillingBuffer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.util.CloseableInputProvider;
 import org.apache.flink.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
index ff397c2..09f4288 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/GroupCombineChainedDriver.java
@@ -28,7 +28,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.RegularPactTask;
 import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
 import org.apache.flink.runtime.operators.sort.InMemorySorter;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
index 4116145..75c1eed 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/chaining/SynchronousChainedCombineDriver.java
@@ -31,7 +31,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.RegularPactTask;
 import org.apache.flink.runtime.operators.sort.FixedLengthRecordSorter;
 import org.apache.flink.runtime.operators.sort.InMemorySorter;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
index ff6548e..5a0c6cc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
@@ -30,7 +30,7 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
 import org.apache.flink.runtime.util.IntArrayList;
 import org.apache.flink.runtime.util.LongArrayList;
 import org.apache.flink.runtime.util.MathUtils;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
index 08b6191..3b12c68 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashMatchIteratorBase.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 
 import java.util.List;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
index b899acd..7baaee7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/HashPartition.java
@@ -35,8 +35,8 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.runtime.util.MathUtils;
 import org.apache.flink.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
index ffb66fc..7554bc3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/InMemoryPartition.java
@@ -28,9 +28,9 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentSource;
 import org.apache.flink.core.memory.SeekableDataInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
 
 /**
  * In-memory partition with overflow buckets for {@link CompactingHashTable}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
index 5000dab..dbdb5b2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstHashMatchIterator.java
@@ -26,8 +26,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
index af1626a..b51c3b1 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildFirstReOpenableHashMatchIterator.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
index 83952c9..26dba7b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondHashMatchIterator.java
@@ -26,8 +26,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
index 029be5f..92b0fff 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/NonReusingBuildSecondReOpenableHashMatchIterator.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.util.MutableObjectIterator;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
index b4aaa95..65dfd89 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstHashMatchIterator.java
@@ -29,8 +29,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
index 714a1f5..5635865 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildFirstReOpenableHashMatchIterator.java
@@ -28,8 +28,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.util.MutableObjectIterator;
 
 public class ReusingBuildFirstReOpenableHashMatchIterator<V1, V2, O> extends ReusingBuildFirstHashMatchIterator<V1, V2, O> {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
index b7c3e29..156f259 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondHashMatchIterator.java
@@ -29,8 +29,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
index 4b4cdf5..a0791fe 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/ReusingBuildSecondReOpenableHashMatchIterator.java
@@ -28,8 +28,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.util.MutableObjectIterator;
 
 public class ReusingBuildSecondReOpenableHashMatchIterator<V1, V2, O> extends ReusingBuildSecondHashMatchIterator<V1, V2, O> {

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
index 730d19a..859757f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/AbstractBlockResettableIterator.java
@@ -30,9 +30,9 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.RandomAccessInputView;
 import org.apache.flink.runtime.io.disk.SimpleCollectingOutputView;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.MemoryBlockIterator;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
index abce462..124ba55 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/BlockResettableMutableObjectIterator.java
@@ -25,8 +25,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.ResettableMutableObjectIterator;
 import org.apache.flink.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
index 9d581ce..408d6e4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/NonReusingBlockResettableIterator.java
@@ -27,8 +27,8 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.ResettableIterator;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
index baa0fb2..ff58092 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/ReusingBlockResettableIterator.java
@@ -21,8 +21,8 @@ package org.apache.flink.runtime.operators.resettable;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
index e5a4501..c59b52a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableIterator.java
@@ -33,9 +33,9 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.SpillingBuffer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.ResettableIterator;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
index 5467ae9..12d5400 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/resettable/SpillingResettableMutableObjectIterator.java
@@ -31,9 +31,9 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.SpillingBuffer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.ResettableMutableObjectIterator;
 import org.apache.flink.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
index e9ccf52..db56227 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeInnerJoinIterator.java
@@ -24,8 +24,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
index 68e0d47..a6bd4d0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeIterator.java
@@ -25,8 +25,8 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.operators.resettable.NonReusingBlockResettableIterator;
 import org.apache.flink.runtime.operators.resettable.SpillingResettableIterator;
 import org.apache.flink.runtime.operators.util.JoinTaskIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
index f34639f..d109cf8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/AbstractMergeOuterJoinIterator.java
@@ -24,8 +24,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.util.Collector;
 import org.apache.flink.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
index 8da9413..f662a7e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningUnilateralSortMerger.java
@@ -30,8 +30,8 @@ import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
 import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
 import org.apache.flink.util.Collector;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
index cd982c4..e932592 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/FixedLengthRecordSorter.java
@@ -28,8 +28,8 @@ import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.util.MutableObjectIterator;
 
 /**
@@ -259,11 +259,11 @@ public final class FixedLengthRecordSorter<T> implements InMemorySorter<T> {
 	//                           Access Utilities
 	// ------------------------------------------------------------------------
 	
-	private final boolean memoryAvailable() {
+	private boolean memoryAvailable() {
 		return !this.freeMemory.isEmpty();
 	}
 	
-	private final MemorySegment nextMemorySegment() {
+	private MemorySegment nextMemorySegment() {
 		return this.freeMemory.remove(this.freeMemory.size() - 1);
 	}
 
@@ -282,7 +282,7 @@ public final class FixedLengthRecordSorter<T> implements InMemorySorter<T> {
 		final MemorySegment segI = this.sortBuffer.get(bufferNumI);
 		final MemorySegment segJ = this.sortBuffer.get(bufferNumJ);
 		
-		int val = MemorySegment.compare(segI, segJ, segmentOffsetI, segmentOffsetJ, this.numKeyBytes);
+		int val = segI.compare(segJ, segmentOffsetI, segmentOffsetJ, this.numKeyBytes);
 		return this.useNormKeyUninverted ? val : -val;
 	}
 
@@ -297,7 +297,7 @@ public final class FixedLengthRecordSorter<T> implements InMemorySorter<T> {
 		final MemorySegment segI = this.sortBuffer.get(bufferNumI);
 		final MemorySegment segJ = this.sortBuffer.get(bufferNumJ);
 		
-		MemorySegment.swapBytes(segI, segJ, this.swapBuffer, segmentOffsetI, segmentOffsetJ, this.recordSize);
+		segI.swapBytes(this.swapBuffer, segJ, segmentOffsetI, segmentOffsetJ, this.recordSize);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
index ce63021..e4a99fb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/LargeRecordHandler.java
@@ -44,7 +44,7 @@ import org.apache.flink.runtime.io.disk.SeekableFileChannelInputView;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.types.NullKeyFieldException;
 import org.apache.flink.util.MutableObjectIterator;
 import org.slf4j.Logger;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
index 644084c..d208398 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeInnerJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.KeyGroupedIterator;
 import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
index 33510d3..db47f16 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NonReusingMergeOuterJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.KeyGroupedIterator;
 import org.apache.flink.runtime.util.NonReusingKeyGroupedIterator;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
index fe87788..c05e518 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/NormalizedKeySorter.java
@@ -29,7 +29,7 @@ import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.runtime.io.disk.RandomAccessInputView;
 import org.apache.flink.runtime.io.disk.SimpleCollectingOutputView;
 import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
-import org.apache.flink.runtime.memorymanager.ListMemorySegmentSource;
+import org.apache.flink.runtime.memory.ListMemorySegmentSource;
 import org.apache.flink.util.MutableObjectIterator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -322,7 +322,7 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
 	//                           Access Utilities
 	// ------------------------------------------------------------------------
 	
-	private final long readPointer(int logicalPosition) {
+	private long readPointer(int logicalPosition) {
 		if (logicalPosition < 0 | logicalPosition >= this.numRecords) {
 			throw new IndexOutOfBoundsException();
 		}
@@ -333,17 +333,17 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
 		return (this.sortIndex.get(bufferNum).getLong(segmentOffset * this.indexEntrySize)) & POINTER_MASK;
 	}
 	
-	private final T getRecordFromBuffer(T reuse, long pointer) throws IOException {
+	private T getRecordFromBuffer(T reuse, long pointer) throws IOException {
 		this.recordBuffer.setReadPosition(pointer);
 		return this.serializer.deserialize(reuse, this.recordBuffer);
 	}
 
-	private final T getRecordFromBuffer(long pointer) throws IOException {
+	private T getRecordFromBuffer(long pointer) throws IOException {
 		this.recordBuffer.setReadPosition(pointer);
 		return this.serializer.deserialize(this.recordBuffer);
 	}
 	
-	private final int compareRecords(long pointer1, long pointer2) {
+	private int compareRecords(long pointer1, long pointer2) {
 		this.recordBuffer.setReadPosition(pointer1);
 		this.recordBufferForComparison.setReadPosition(pointer2);
 		
@@ -354,11 +354,11 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
 		}
 	}
 	
-	private final boolean memoryAvailable() {
+	private boolean memoryAvailable() {
 		return !this.freeMemory.isEmpty();
 	}
 	
-	private final MemorySegment nextMemorySegment() {
+	private MemorySegment nextMemorySegment() {
 		return this.freeMemory.remove(this.freeMemory.size() - 1);
 	}
 
@@ -377,7 +377,7 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
 		final MemorySegment segI = this.sortIndex.get(bufferNumI);
 		final MemorySegment segJ = this.sortIndex.get(bufferNumJ);
 		
-		int val = MemorySegment.compare(segI, segJ, segmentOffsetI + OFFSET_LEN, segmentOffsetJ + OFFSET_LEN, this.numKeyBytes);
+		int val = segI.compare(segJ, segmentOffsetI + OFFSET_LEN, segmentOffsetJ + OFFSET_LEN, this.numKeyBytes);
 		
 		if (val != 0 || this.normalizedKeyFullyDetermines) {
 			return this.useNormKeyUninverted ? val : -val;
@@ -400,7 +400,7 @@ public final class NormalizedKeySorter<T> implements InMemorySorter<T> {
 		final MemorySegment segI = this.sortIndex.get(bufferNumI);
 		final MemorySegment segJ = this.sortIndex.get(bufferNumJ);
 		
-		MemorySegment.swapBytes(segI, segJ, this.swapBuffer, segmentOffsetI, segmentOffsetJ, this.indexEntrySize);
+		segI.swapBytes(this.swapBuffer, segJ, segmentOffsetI, segmentOffsetJ, this.indexEntrySize);
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
index 3a1a17a..6b8fcf9 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeInnerJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.KeyGroupedIterator;
 import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
index ce7bab4..8382b86 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/ReusingMergeOuterJoinIterator.java
@@ -23,8 +23,8 @@ import org.apache.flink.api.common.typeutils.TypePairComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.KeyGroupedIterator;
 import org.apache.flink.runtime.util.ReusingKeyGroupedIterator;
 import org.apache.flink.util.MutableObjectIterator;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
index 388b7b9..fd1062d 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/UnilateralSortMerger.java
@@ -46,8 +46,8 @@ import org.apache.flink.runtime.io.disk.iomanager.ChannelWriterOutputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel.ID;
 import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.util.EmptyMutableObjectIterator;
 import org.apache.flink.util.MutableObjectIterator;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
index 6152797..ac3668b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/CoGroupTaskIterator.java
@@ -21,7 +21,7 @@ package org.apache.flink.runtime.operators.util;
 
 import java.io.IOException;
 
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
 
 /**
  * Interface describing the methods that have to be implemented by local strategies for the CoGroup Pact.

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
index 1f1d625..197eaf8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/util/JoinTaskIterator.java
@@ -23,7 +23,7 @@ package org.apache.flink.runtime.operators.util;
 import java.io.IOException;
 
 import org.apache.flink.api.common.functions.FlatJoinFunction;
-import org.apache.flink.runtime.memorymanager.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryAllocationException;
 import org.apache.flink.util.Collector;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
index 07cea33..567f051 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/RuntimeEnvironment.java
@@ -31,7 +31,7 @@ import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
 import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint;
 import org.apache.flink.runtime.state.StateHandle;
 import org.apache.flink.util.SerializedValue;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index a19a57a..0d1dd13 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -49,7 +49,7 @@ import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointNotificationOperator;
 import org.apache.flink.runtime.jobgraph.tasks.CheckpointedOperator;
 import org.apache.flink.runtime.jobgraph.tasks.OperatorStateCarrier;
-import org.apache.flink.runtime.memorymanager.MemoryManager;
+import org.apache.flink.runtime.memory.MemoryManager;
 import org.apache.flink.runtime.messages.TaskManagerMessages.FatalError;
 import org.apache.flink.runtime.messages.TaskMessages.TaskInFinalState;
 import org.apache.flink.runtime.messages.TaskMessages.UpdateTaskExecutionState;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
index f243027..b8519b0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
@@ -116,7 +116,24 @@ public class EnvironmentInformation {
 	 * @return The maximum JVM heap size, in bytes.
 	 */
 	public static long getMaxJvmHeapMemory() {
-		return Runtime.getRuntime().maxMemory();
+		long maxMemory = Runtime.getRuntime().maxMemory();
+
+		if (maxMemory == Long.MAX_VALUE) {
+			// amount of free memory unknown
+			try {
+				// workaround for Oracle JDK
+				OperatingSystemMXBean operatingSystemMXBean = ManagementFactory.getOperatingSystemMXBean();
+				Class<?> clazz = Class.forName("com.sun.management.OperatingSystemMXBean");
+				Method method = clazz.getMethod("getTotalPhysicalMemorySize");
+				maxMemory = (Long) method.invoke(operatingSystemMXBean) / 4;
+			}
+			catch (Throwable e) {
+				throw new RuntimeException("Could not determine the amount of free memory.\n" +
+						"Please set the maximum memory for the JVM, e.g. -Xmx512M for 512 megabytes.");
+			}
+		}
+		
+		return maxMemory;
 	}
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
index 5bcda60..065211c 100644
--- a/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
+++ b/flink-runtime/src/main/scala/org/apache/flink/runtime/taskmanager/NetworkEnvironmentConfiguration.scala
@@ -18,12 +18,14 @@
 
 package org.apache.flink.runtime.taskmanager
 
+import org.apache.flink.core.memory.MemoryType
 import org.apache.flink.runtime.io.disk.iomanager.IOManager.IOMode
 import org.apache.flink.runtime.io.network.netty.NettyConfig
 
 case class NetworkEnvironmentConfiguration(
   numNetworkBuffers: Int,
   networkBufferSize: Int,
+  memoryType: MemoryType,
   ioMode: IOMode,
   nettyConfig: Option[NettyConfig] = None,
-  partitionRequestInitialAndMaxBackoff: Tuple2[Integer, Integer] = (500, 3000))
+  partitionRequestInitialAndMaxBackoff: (Integer, Integer) = (500, 3000))


[02/15] flink git commit: [FLINK-2636] [streaming] Create common type StreamElement for StreamRecord and Watermark

Posted by se...@apache.org.
[FLINK-2636] [streaming] Create common type StreamElement for StreamRecord and Watermark


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

Branch: refs/heads/master
Commit: c09d14a9d836d099c32639180dd58216ae6149a4
Parents: 655a891
Author: Stephan Ewen <se...@apache.org>
Authored: Tue Sep 8 15:21:23 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Tue Sep 8 20:58:05 2015 +0200

----------------------------------------------------------------------
 .../api/common/typeutils/TypeSerializer.java    |  1 -
 .../streaming/api/watermark/Watermark.java      | 24 ++----
 .../runtime/io/RecordWriterOutput.java          | 13 ++--
 .../runtime/io/StreamInputProcessor.java        | 29 ++++---
 .../runtime/io/StreamTwoInputProcessor.java     | 63 ++++++++--------
 .../MultiplexingStreamRecordSerializer.java     | 79 +++++++++++---------
 .../runtime/streamrecord/StreamElement.java     | 62 +++++++++++++++
 .../runtime/streamrecord/StreamRecord.java      |  4 +-
 .../consumer/StreamTestSingleInputGate.java     |  5 +-
 .../api/operators/StreamFilterTest.java         |  1 -
 .../api/operators/StreamGroupedFoldTest.java    |  2 -
 .../api/operators/StreamGroupedReduceTest.java  |  1 -
 .../streaming/api/operators/StreamMapTest.java  |  1 -
 .../api/operators/StreamProjectTest.java        |  3 +-
 .../api/operators/co/CoStreamFlatMapTest.java   |  5 --
 .../runtime/tasks/StreamTaskTestHarness.java    |  6 +-
 16 files changed, 173 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
index 542b059..45b0669 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
@@ -16,7 +16,6 @@
  * limitations under the License.
  */
 
-
 package org.apache.flink.api.common.typeutils;
 
 import java.io.IOException;

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
index 1d88fe2..163791e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/watermark/Watermark.java
@@ -17,6 +17,8 @@
  */
 package org.apache.flink.streaming.api.watermark;
 
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+
 /**
  * A Watermark tells operators that receive it that no elements with a timestamp older or equal
  * to the watermark timestamp should arrive at the operator. Watermarks are emitted at the
@@ -31,11 +33,11 @@ package org.apache.flink.streaming.api.watermark;
  * In some cases a watermark is only a heuristic and operators should be able to deal with
  * late elements. They can either discard those or update the result and emit updates/retractions
  * to downstream operations.
- *
  */
-public class Watermark {
+public class Watermark extends StreamElement {
 
-	private long timestamp;
+	/** The timestamp of the watermark */
+	private final long timestamp;
 
 	/**
 	 * Creates a new watermark with the given timestamp.
@@ -53,16 +55,8 @@ public class Watermark {
 
 	@Override
 	public boolean equals(Object o) {
-		if (this == o) {
-			return true;
-		}
-		if (o == null || getClass() != o.getClass()) {
-			return false;
-		}
-
-		Watermark watermark = (Watermark) o;
-
-		return timestamp == watermark.timestamp;
+		return this == o ||
+				o != null && o.getClass() == Watermark.class && ((Watermark) o).timestamp == this.timestamp;
 	}
 
 	@Override
@@ -72,8 +66,6 @@ public class Watermark {
 
 	@Override
 	public String toString() {
-		return "Watermark{" +
-				"timestamp=" + timestamp +
-				'}';
+		return "Watermark @ " + timestamp;
 	}
 }

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
index 7048464..34e5800 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/RecordWriterOutput.java
@@ -24,6 +24,7 @@ import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
 import org.apache.flink.runtime.plugable.SerializationDelegate;
 import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -36,9 +37,9 @@ import static com.google.common.base.Preconditions.checkNotNull;
  */
 public class RecordWriterOutput<OUT> implements Output<StreamRecord<OUT>> {
 
-	private StreamRecordWriter<SerializationDelegate<Object>> recordWriter;
+	private StreamRecordWriter<SerializationDelegate<StreamElement>> recordWriter;
 	
-	private SerializationDelegate<Object> serializationDelegate;
+	private SerializationDelegate<StreamElement> serializationDelegate;
 
 	
 	@SuppressWarnings("unchecked")
@@ -51,19 +52,19 @@ public class RecordWriterOutput<OUT> implements Output<StreamRecord<OUT>> {
 		
 		// generic hack: cast the writer to generic Object type so we can use it 
 		// with multiplexed records and watermarks
-		this.recordWriter = (StreamRecordWriter<SerializationDelegate<Object>>) 
+		this.recordWriter = (StreamRecordWriter<SerializationDelegate<StreamElement>>) 
 				(StreamRecordWriter<?>) recordWriter;
 
-		TypeSerializer<Object> outRecordSerializer;
+		TypeSerializer<StreamElement> outRecordSerializer;
 		if (enableWatermarkMultiplexing) {
 			outRecordSerializer = new MultiplexingStreamRecordSerializer<OUT>(outSerializer);
 		} else {
-			outRecordSerializer = (TypeSerializer<Object>)
+			outRecordSerializer = (TypeSerializer<StreamElement>)
 					(TypeSerializer<?>) new StreamRecordSerializer<OUT>(outSerializer);
 		}
 
 		if (outSerializer != null) {
-			serializationDelegate = new SerializationDelegate<Object>(outRecordSerializer);
+			serializationDelegate = new SerializationDelegate<StreamElement>(outRecordSerializer);
 		}
 	}
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index de021ff..cc91d63 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -36,6 +36,7 @@ import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
 import org.apache.flink.runtime.util.event.EventListener;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -53,9 +54,9 @@ import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
  */
 public class StreamInputProcessor<IN> extends AbstractReader implements StreamingReader {
 	
-	private final RecordDeserializer<DeserializationDelegate<Object>>[] recordDeserializers;
+	private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
 
-	private RecordDeserializer<DeserializationDelegate<Object>> currentRecordDeserializer;
+	private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
 
 	// We need to keep track of the channel from which a buffer came, so that we can
 	// appropriately map the watermarks to input channels
@@ -68,9 +69,9 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 	private final long[] watermarks;
 	private long lastEmittedWatermark;
 
-	private final DeserializationDelegate<Object> deserializationDelegate;
+	private final DeserializationDelegate<StreamElement> deserializationDelegate;
 
-	@SuppressWarnings("unchecked")
+	@SuppressWarnings({"unchecked", "rawtypes"})
 	public StreamInputProcessor(InputGate[] inputGates, TypeSerializer<IN> inputSerializer,
 								EventListener<CheckpointBarrier> checkpointListener,
 								CheckpointingMode checkpointMode,
@@ -95,10 +96,10 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 		
 		if (enableWatermarkMultiplexing) {
 			MultiplexingStreamRecordSerializer<IN> ser = new MultiplexingStreamRecordSerializer<IN>(inputSerializer);
-			this.deserializationDelegate = new NonReusingDeserializationDelegate<Object>(ser);
+			this.deserializationDelegate = new NonReusingDeserializationDelegate<StreamElement>(ser);
 		} else {
 			StreamRecordSerializer<IN> ser = new StreamRecordSerializer<IN>(inputSerializer);
-			this.deserializationDelegate = (NonReusingDeserializationDelegate<Object>)
+			this.deserializationDelegate = (NonReusingDeserializationDelegate<StreamElement>)
 					(NonReusingDeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN>>(ser);
 		}
 		
@@ -106,7 +107,7 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 		this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()];
 		
 		for (int i = 0; i < recordDeserializers.length; i++) {
-			recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<Object>>();
+			recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<StreamElement>>();
 		}
 
 		watermarks = new long[inputGate.getNumberOfInputChannels()];
@@ -132,18 +133,15 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 				}
 
 				if (result.isFullRecord()) {
-					Object recordOrWatermark = deserializationDelegate.getInstance();
+					StreamElement recordOrWatermark = deserializationDelegate.getInstance();
 
-					if (recordOrWatermark instanceof Watermark) {
-						Watermark mark = (Watermark) recordOrWatermark;
-						long watermarkMillis = mark.getTimestamp();
+					if (recordOrWatermark.isWatermark()) {
+						long watermarkMillis = recordOrWatermark.asWatermark().getTimestamp();
 						if (watermarkMillis > watermarks[currentChannel]) {
 							watermarks[currentChannel] = watermarkMillis;
 							long newMinWatermark = Long.MAX_VALUE;
 							for (long watermark : watermarks) {
-								if (watermark < newMinWatermark) {
-									newMinWatermark = watermark;
-								}
+								newMinWatermark = Math.min(watermark, newMinWatermark);
 							}
 							if (newMinWatermark > lastEmittedWatermark) {
 								lastEmittedWatermark = newMinWatermark;
@@ -154,8 +152,7 @@ public class StreamInputProcessor<IN> extends AbstractReader implements Streamin
 					}
 					else {
 						// now we can do the actual processing
-						@SuppressWarnings("unchecked")
-						StreamRecord<IN> record = (StreamRecord<IN>) deserializationDelegate.getInstance();
+						StreamRecord<IN> record = recordOrWatermark.asRecord();
 						StreamingRuntimeContext ctx = streamOperator.getRuntimeContext();
 						if (ctx != null) {
 							ctx.setNextInput(record);

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
index e0af729..7dffa71 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamTwoInputProcessor.java
@@ -35,6 +35,7 @@ import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
 import org.apache.flink.runtime.util.event.EventListener;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -44,6 +45,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Collection;
 
 /**
@@ -61,9 +63,9 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 	@SuppressWarnings("unused")
 	private static final Logger LOG = LoggerFactory.getLogger(StreamTwoInputProcessor.class);
 
-	private final RecordDeserializer<DeserializationDelegate<Object>>[] recordDeserializers;
+	private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers;
 
-	private RecordDeserializer<DeserializationDelegate<Object>> currentRecordDeserializer;
+	private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer;
 
 	// We need to keep track of the channel from which a buffer came, so that we can
 	// appropriately map the watermarks to input channels
@@ -81,8 +83,8 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 
 	private final int numInputChannels1;
 
-	private final DeserializationDelegate<Object> deserializationDelegate1;
-	private final DeserializationDelegate<Object> deserializationDelegate2;
+	private final DeserializationDelegate<StreamElement> deserializationDelegate1;
+	private final DeserializationDelegate<StreamElement> deserializationDelegate2;
 
 	@SuppressWarnings({"unchecked", "rawtypes"})
 	public StreamTwoInputProcessor(
@@ -113,21 +115,21 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 		
 		if (enableWatermarkMultiplexing) {
 			MultiplexingStreamRecordSerializer<IN1> ser = new MultiplexingStreamRecordSerializer<IN1>(inputSerializer1);
-			this.deserializationDelegate1 = new NonReusingDeserializationDelegate<Object>(ser);
+			this.deserializationDelegate1 = new NonReusingDeserializationDelegate<StreamElement>(ser);
 		}
 		else {
 			StreamRecordSerializer<IN1> ser = new StreamRecordSerializer<IN1>(inputSerializer1);
-			this.deserializationDelegate1 = (DeserializationDelegate<Object>)
+			this.deserializationDelegate1 = (DeserializationDelegate<StreamElement>)
 					(DeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN1>>(ser);
 		}
 		
 		if (enableWatermarkMultiplexing) {
 			MultiplexingStreamRecordSerializer<IN2> ser = new MultiplexingStreamRecordSerializer<IN2>(inputSerializer2);
-			this.deserializationDelegate2 = new NonReusingDeserializationDelegate<Object>(ser);
+			this.deserializationDelegate2 = new NonReusingDeserializationDelegate<StreamElement>(ser);
 		}
 		else {
 			StreamRecordSerializer<IN2> ser = new StreamRecordSerializer<IN2>(inputSerializer2);
-			this.deserializationDelegate2 = (DeserializationDelegate<Object>)
+			this.deserializationDelegate2 = (DeserializationDelegate<StreamElement>)
 					(DeserializationDelegate<?>) new NonReusingDeserializationDelegate<StreamRecord<IN2>>(ser);
 		}
 
@@ -135,7 +137,7 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 		this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[inputGate.getNumberOfInputChannels()];
 		
 		for (int i = 0; i < recordDeserializers.length; i++) {
-			recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<Object>>();
+			recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<DeserializationDelegate<StreamElement>>();
 		}
 
 		// determine which unioned channels belong to input 1 and which belong to input 2
@@ -148,15 +150,11 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 		int numInputChannels2 = inputGate.getNumberOfInputChannels() - numInputChannels1;
 
 		watermarks1 = new long[numInputChannels1];
-		for (int i = 0; i < numInputChannels1; i++) {
-			watermarks1[i] = Long.MIN_VALUE;
-		}
+		Arrays.fill(watermarks1, Long.MIN_VALUE);
 		lastEmittedWatermark1 = Long.MIN_VALUE;
 
 		watermarks2 = new long[numInputChannels2];
-		for (int i = 0; i < numInputChannels2; i++) {
-			watermarks2[i] = Long.MIN_VALUE;
-		}
+		Arrays.fill(watermarks2, Long.MIN_VALUE);
 		lastEmittedWatermark2 = Long.MIN_VALUE;
 	}
 
@@ -182,22 +180,25 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 
 				if (result.isFullRecord()) {
 					if (currentChannel < numInputChannels1) {
-						Object recordOrWatermark = deserializationDelegate1.getInstance();
-						if (recordOrWatermark instanceof Watermark) {
+						StreamElement recordOrWatermark = deserializationDelegate1.getInstance();
+						if (recordOrWatermark.isWatermark()) {
 							handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel);
 							continue;
-						} else {
-							streamOperator.processElement1((StreamRecord<IN1>) deserializationDelegate1.getInstance());
+						}
+						else {
+							streamOperator.processElement1(recordOrWatermark.<IN1>asRecord());
 							return true;
 
 						}
-					} else {
-						Object recordOrWatermark = deserializationDelegate2.getInstance();
-						if (recordOrWatermark instanceof Watermark) {
-							handleWatermark(streamOperator, (Watermark) recordOrWatermark, currentChannel);
+					}
+					else {
+						StreamElement recordOrWatermark = deserializationDelegate2.getInstance();
+						if (recordOrWatermark.isWatermark()) {
+							handleWatermark(streamOperator, recordOrWatermark.asWatermark(), currentChannel);
 							continue;
-						} else {
-							streamOperator.processElement2((StreamRecord<IN2>) deserializationDelegate2.getInstance());
+						}
+						else {
+							streamOperator.processElement2(recordOrWatermark.<IN2>asRecord());
 							return true;
 						}
 					}
@@ -234,10 +235,8 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 			if (watermarkMillis > watermarks1[channelIndex]) {
 				watermarks1[channelIndex] = watermarkMillis;
 				long newMinWatermark = Long.MAX_VALUE;
-				for (long aWatermarks1 : watermarks1) {
-					if (aWatermarks1 < newMinWatermark) {
-						newMinWatermark = aWatermarks1;
-					}
+				for (long wm : watermarks1) {
+					newMinWatermark = Math.min(wm, newMinWatermark);
 				}
 				if (newMinWatermark > lastEmittedWatermark1) {
 					lastEmittedWatermark1 = newMinWatermark;
@@ -250,10 +249,8 @@ public class StreamTwoInputProcessor<IN1, IN2> extends AbstractReader implements
 			if (watermarkMillis > watermarks2[channelIndex]) {
 				watermarks2[channelIndex] = watermarkMillis;
 				long newMinWatermark = Long.MAX_VALUE;
-				for (long aWatermarks2 : watermarks2) {
-					if (aWatermarks2 < newMinWatermark) {
-						newMinWatermark = aWatermarks2;
-					}
+				for (long wm : watermarks2) {
+					newMinWatermark = Math.min(wm, newMinWatermark);
 				}
 				if (newMinWatermark > lastEmittedWatermark2) {
 					lastEmittedWatermark2 = newMinWatermark;

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
index 075c4fc..156e5d6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java
@@ -27,22 +27,22 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import java.io.IOException;
 
 /**
- * Serializer for {@link StreamRecord} and {@link Watermark}. This does not behave like a normal
+ * Serializer for {@link StreamRecord} and {@link org.apache.flink.streaming.api.watermark.Watermark}. This does not behave like a normal
  * {@link TypeSerializer}, instead, this is only used at the
  * {@link org.apache.flink.streaming.runtime.tasks.StreamTask} level for transmitting
- * {@link StreamRecord StreamRecords} and {@link Watermark Watermarks}. This serializer
+ * {@link StreamRecord StreamRecords} and {@link org.apache.flink.streaming.api.watermark.Watermark Watermarks}. This serializer
  * can handle both of them, therefore it returns {@link Object} the result has
  * to be cast to the correct type.
  *
  * @param <T> The type of value in the {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}
  */
-public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<Object> {
+public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<StreamElement> {
 
 	private static final long serialVersionUID = 1L;
 
 	private static final long IS_WATERMARK = Long.MIN_VALUE;
 	
-	protected final TypeSerializer<T> typeSerializer;
+	private final TypeSerializer<T> typeSerializer;
 
 	
 	public MultiplexingStreamRecordSerializer(TypeSerializer<T> serializer) {
@@ -59,87 +59,94 @@ public final class MultiplexingStreamRecordSerializer<T> extends TypeSerializer<
 	}
 
 	@Override
-	public TypeSerializer<Object> duplicate() {
-		return this;
+	public TypeSerializer<StreamElement> duplicate() {
+		TypeSerializer<T> copy = typeSerializer.duplicate();
+		return (copy == typeSerializer) ? this : new MultiplexingStreamRecordSerializer<T>(copy);
 	}
 
 	@Override
-	public Object createInstance() {
+	public StreamRecord<T> createInstance() {
 		return new StreamRecord<T>(typeSerializer.createInstance(), 0L);
 	}
 
 	@Override
-	@SuppressWarnings("unchecked")
-	public Object copy(Object from) {
+	public StreamElement copy(StreamElement from) {
 		// we can reuse the timestamp since Instant is immutable
-		if (from instanceof StreamRecord) {
-			StreamRecord<T> fromRecord = (StreamRecord<T>) from;
+		if (from.isRecord()) {
+			StreamRecord<T> fromRecord = from.asRecord();
 			return new StreamRecord<T>(typeSerializer.copy(fromRecord.getValue()), fromRecord.getTimestamp());
-		} else if (from instanceof Watermark) {
+		}
+		else if (from.isWatermark()) {
 			// is immutable
 			return from;
-		} else {
+		}
+		else {
 			throw new RuntimeException("Cannot copy " + from);
 		}
 	}
 
 	@Override
-	@SuppressWarnings("unchecked")
-	public Object copy(Object from, Object reuse) {
-		if (from instanceof StreamRecord && reuse instanceof StreamRecord) {
-			StreamRecord<T> fromRecord = (StreamRecord<T>) from;
-			StreamRecord<T> reuseRecord = (StreamRecord<T>) reuse;
+	public StreamElement copy(StreamElement from, StreamElement reuse) {
+		if (from.isRecord() && reuse.isRecord()) {
+			StreamRecord<T> fromRecord = from.asRecord();
+			StreamRecord<T> reuseRecord = reuse.asRecord();
 
-			reuseRecord.replace(typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue()), fromRecord.getTimestamp());
+			T valueCopy = typeSerializer.copy(fromRecord.getValue(), reuseRecord.getValue());
+			reuseRecord.replace(valueCopy, fromRecord.getTimestamp());
 			return reuse;
-		} else if (from instanceof Watermark) {
+		}
+		else if (from.isWatermark()) {
 			// is immutable
 			return from;
-		} else {
-			throw new RuntimeException("Cannot copy " + from);
+		}
+		else {
+			throw new RuntimeException("Cannot copy " + from + " -> " + reuse);
 		}
 	}
 
 	@Override
 	public int getLength() {
-		return 0;
+		return -1;
 	}
 
 	@Override
-	@SuppressWarnings("unchecked")
-	public void serialize(Object value, DataOutputView target) throws IOException {
-		if (value instanceof StreamRecord) {
-			StreamRecord<T> record = (StreamRecord<T>) value;
+	public void serialize(StreamElement value, DataOutputView target) throws IOException {
+		if (value.isRecord()) {
+			StreamRecord<T> record = value.asRecord();
 			target.writeLong(record.getTimestamp());
 			typeSerializer.serialize(record.getValue(), target);
-		} else if (value instanceof Watermark) {
+		}
+		else if (value.isWatermark()) {
 			target.writeLong(IS_WATERMARK);
-			target.writeLong(((Watermark) value).getTimestamp());
+			target.writeLong(value.asWatermark().getTimestamp());
+		}
+		else {
+			throw new RuntimeException();
 		}
 	}
 	
 	@Override
-	public Object deserialize(DataInputView source) throws IOException {
+	public StreamElement deserialize(DataInputView source) throws IOException {
 		long millis = source.readLong();
 
 		if (millis == IS_WATERMARK) {
 			return new Watermark(source.readLong());
-		} else {
+		}
+		else {
 			T element = typeSerializer.deserialize(source);
 			return new StreamRecord<T>(element, millis);
 		}
 	}
 
 	@Override
-	@SuppressWarnings("unchecked")
-	public Object deserialize(Object reuse, DataInputView source) throws IOException {
+	public StreamElement deserialize(StreamElement reuse, DataInputView source) throws IOException {
 		long millis = source.readLong();
 
 		if (millis == IS_WATERMARK) {
 			return new Watermark(source.readLong());
-
-		} else {
-			StreamRecord<T> reuseRecord = (StreamRecord<T>) reuse;
+		}
+		else {
+			StreamRecord<T> reuseRecord = reuse.asRecord();
 			T element = typeSerializer.deserialize(reuseRecord.getValue(), source);
 			reuseRecord.replace(element, millis);
 			return reuse;

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
new file mode 100644
index 0000000..80df72e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElement.java
@@ -0,0 +1,62 @@
+/*
+ * 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.streaming.runtime.streamrecord;
+
+import org.apache.flink.streaming.api.watermark.Watermark;
+
+/**
+ * An element in a data stream. Can be a record or a Watermark.
+ */
+public abstract class StreamElement {
+	
+	/**
+	 * Checks whether this element is a watermark.
+	 * @return True, if this element is a watermark, false otherwise.
+	 */
+	public final boolean isWatermark() {
+		return getClass() == Watermark.class;
+	}
+
+	/**
+	 * Checks whether this element is a record.
+	 * @return True, if this element is a record, false otherwise.
+	 */
+	public final boolean isRecord() {
+		return getClass() == StreamRecord.class;
+	}
+
+	/**
+	 * Casts this element into a StreamRecord.
+	 * @return This element as a stream record.
+	 * @throws java.lang.ClassCastException Thrown, if this element is actually not a stream record.
+	 */
+	@SuppressWarnings("unchecked")
+	public final <E> StreamRecord<E> asRecord() {
+		return (StreamRecord<E>) this;
+	}
+
+	/**
+	 * Casts this element into a Watermark.
+	 * @return This element as a Watermark.
+	 * @throws java.lang.ClassCastException Thrown, if this element is actually not a Watermark.
+	 */
+	public final Watermark asWatermark() {
+		return (Watermark) this;
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
index 92ce66f..348b974 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamRecord.java
@@ -22,10 +22,12 @@ package org.apache.flink.streaming.runtime.streamrecord;
  * 
  * @param <T> The type encapsulated with the stream record.
  */
-public class StreamRecord<T> {
+public class StreamRecord<T> extends StreamElement {
 	
+	/** The actual value held by this record */
 	private T value;
 	
+	/** The timestamp of the record */
 	private long timestamp;
 
 	/**

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
index b78ec44..1187fe6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/StreamTestSingleInputGate.java
@@ -20,7 +20,6 @@
 package org.apache.flink.runtime.io.network.partition.consumer;
 
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.core.memory.HeapMemorySegment;
 import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
@@ -32,6 +31,7 @@ import org.apache.flink.runtime.io.network.buffer.BufferRecycler;
 import org.apache.flink.runtime.jobgraph.IntermediateResultPartitionID;
 import org.apache.flink.runtime.plugable.SerializationDelegate;
 import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.stubbing.Answer;
 
@@ -85,7 +85,8 @@ public class StreamTestSingleInputGate<T> extends TestSingleInputGate {
 		for (int i = 0; i < numInputChannels; i++) {
 			final int channelIndex = i;
 			final RecordSerializer<SerializationDelegate<Object>> recordSerializer = new SpanningRecordSerializer<SerializationDelegate<Object>>();
-			final SerializationDelegate<Object> delegate = new SerializationDelegate<Object>(new MultiplexingStreamRecordSerializer<T>(serializer));
+			final SerializationDelegate<Object> delegate = (SerializationDelegate<Object>) (SerializationDelegate<?>)
+					new SerializationDelegate<StreamElement>(new MultiplexingStreamRecordSerializer<T>(serializer));
 
 			inputQueues[channelIndex] = new ConcurrentLinkedQueue<InputValue<Object>>();
 			inputChannels[channelIndex] = new TestInputChannel(inputGate, i);

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
index bf4fe40..047aad8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
index cb08e65..dcfe3de 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
@@ -21,7 +21,6 @@ import java.util.concurrent.ConcurrentLinkedQueue;
 
 import org.apache.flink.api.common.functions.FoldFunction;
 import org.apache.flink.api.common.functions.RichFoldFunction;
-import org.apache.flink.api.common.functions.RichReduceFunction;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.api.java.functions.KeySelector;
@@ -31,7 +30,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
index 9e35fa2..a2cd1fd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
@@ -28,7 +28,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
index 4d12492..f0113d1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamMapTest.java
@@ -26,7 +26,6 @@ import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.joda.time.Instant;
 import org.junit.Assert;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
index bb9dad7..14abd18 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamProjectTest.java
@@ -20,7 +20,6 @@ package org.apache.flink.streaming.api.operators;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-import java.io.Serializable;
 import java.util.HashSet;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
@@ -42,7 +41,7 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
 import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
 import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
+
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
index 4c644a9..4dbf7b8 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java
@@ -23,19 +23,14 @@ import java.io.Serializable;
 import java.util.concurrent.ConcurrentLinkedQueue;
 
 import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
 import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.util.NoOpSink;
 import org.apache.flink.streaming.util.TestHarnessUtil;
 import org.apache.flink.streaming.util.TwoInputStreamOperatorTestHarness;
 import org.apache.flink.util.Collector;
 import org.junit.Assert;
-import org.junit.Ignore;
 import org.junit.Test;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/c09d14a9/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index 96dbeab..06fca6b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -35,6 +35,7 @@ import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
 import org.apache.flink.streaming.api.operators.StreamOperator;
 import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner;
 import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
 import org.apache.flink.util.InstantiationUtil;
 import org.junit.Assert;
 
@@ -79,7 +80,7 @@ public class StreamTaskTestHarness<OUT> {
 	private AbstractInvokable task;
 
 	private TypeSerializer<OUT> outputSerializer;
-	private TypeSerializer<Object> outputStreamRecordSerializer;
+	private TypeSerializer<StreamElement> outputStreamRecordSerializer;
 
 	private ConcurrentLinkedQueue<Object> outputList;
 
@@ -119,8 +120,7 @@ public class StreamTaskTestHarness<OUT> {
 	/**
 	 * This must be overwritten for OneInputStreamTask or TwoInputStreamTask test harnesses.
 	 */
-	protected void initializeInputs() throws IOException, InterruptedException {
-	}
+	protected void initializeInputs() throws IOException, InterruptedException {}
 
 	@SuppressWarnings("unchecked")
 	private void initializeOutput() {


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

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
index 35ec3e3..a107e79 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SimpleCollectingOutputView.java
@@ -25,7 +25,7 @@ import java.util.List;
 
 import org.apache.flink.core.memory.MemorySegment;
 import org.apache.flink.core.memory.MemorySegmentSource;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 import org.apache.flink.runtime.util.MathUtils;
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
index 5f9c2cf..397c40c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/SpillingBuffer.java
@@ -30,7 +30,7 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
 import org.apache.flink.runtime.io.disk.iomanager.HeaderlessChannelReaderInputView;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 
 
 /**

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
index b919034..de1ed87 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelReaderInputView.java
@@ -25,7 +25,7 @@ import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
 
 /**
  * A {@link org.apache.flink.core.memory.DataInputView} that is backed by a

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
index 089e10a..64b2ebc 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/disk/iomanager/ChannelWriterOutputView.java
@@ -24,7 +24,7 @@ import java.util.List;
 import java.util.concurrent.LinkedBlockingQueue;
 
 import org.apache.flink.core.memory.MemorySegment;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 
 /**
  * A {@link org.apache.flink.core.memory.DataOutputView} that is backed by a

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
index 9eb4a9a..6e4b564 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java
@@ -107,7 +107,8 @@ public class NetworkEnvironment {
 		// create the network buffers - this is the operation most likely to fail upon
 		// mis-configuration, so we do this first
 		try {
-			networkBufferPool = new NetworkBufferPool(config.numNetworkBuffers(), config.networkBufferSize());
+			networkBufferPool = new NetworkBufferPool(config.numNetworkBuffers(),
+					config.networkBufferSize(), config.memoryType());
 		}
 		catch (Throwable t) {
 			throw new IOException("Cannot allocate network buffer pool: " + t.getMessage(), t);

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
index ec9f4fd..cdd8731 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/AdaptiveSpanningRecordDeserializer.java
@@ -230,21 +230,21 @@ public class AdaptiveSpanningRecordDeserializer<T extends IOReadableWritable> im
 
 		@Override
 		public final short readShort() throws IOException {
-			final short v = this.segment.getShort(this.position);
+			final short v = this.segment.getShortBigEndian(this.position);
 			this.position += 2;
 			return v;
 		}
 
 		@Override
 		public final int readUnsignedShort() throws IOException {
-			final int v = this.segment.getShort(this.position) & 0xffff;
+			final int v = this.segment.getShortBigEndian(this.position) & 0xffff;
 			this.position += 2;
 			return v;
 		}
 
 		@Override
 		public final char readChar() throws IOException  {
-			final char v = this.segment.getChar(this.position);
+			final char v = this.segment.getCharBigEndian(this.position);
 			this.position += 2;
 			return v;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
index b23b83b..a34f8cf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/EventSerializer.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.network.api.serialization;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.event.AbstractEvent;
 import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
@@ -145,8 +146,9 @@ public class EventSerializer {
 	public static Buffer toBuffer(AbstractEvent event) {
 		final ByteBuffer serializedEvent = EventSerializer.toSerializedEvent(event);
 
-		final Buffer buffer = new Buffer(new MemorySegment(serializedEvent.array()),
-				FreeingBufferRecycler.INSTANCE, false);
+		MemorySegment data = MemorySegmentFactory.wrap(serializedEvent.array());
+		
+		final Buffer buffer = new Buffer(data, FreeingBufferRecycler.INSTANCE, false);
 		buffer.setSize(serializedEvent.remaining());
 
 		return buffer;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
index f3e4892..b10e5a8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/serialization/SpillingAdaptiveSpanningRecordDeserializer.java
@@ -262,21 +262,21 @@ public class SpillingAdaptiveSpanningRecordDeserializer<T extends IOReadableWrit
 
 		@Override
 		public final short readShort() throws IOException {
-			final short v = this.segment.getShort(this.position);
+			final short v = this.segment.getShortBigEndian(this.position);
 			this.position += 2;
 			return v;
 		}
 
 		@Override
 		public final int readUnsignedShort() throws IOException {
-			final int v = this.segment.getShort(this.position) & 0xffff;
+			final int v = this.segment.getShortBigEndian(this.position) & 0xffff;
 			this.position += 2;
 			return v;
 		}
 
 		@Override
 		public final char readChar() throws IOException  {
-			final char v = this.segment.getChar(this.position);
+			final char v = this.segment.getCharBigEndian(this.position);
 			this.position += 2;
 			return v;
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
index 23c7ed0..5fb62cb 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/Buffer.java
@@ -84,9 +84,8 @@ public class Buffer {
 	public ByteBuffer getNioBuffer() {
 		synchronized (recycleLock) {
 			ensureNotRecycled();
-
-			// we need to return a copy here to guarantee thread-safety
-			return memorySegment.wrap(0, currentSize).duplicate();
+			// the memory segment returns a distinct buffer every time
+			return memorySegment.wrap(0, currentSize);
 		}
 	}
 	

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
index cb1f118..209d925 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java
@@ -20,15 +20,20 @@ package org.apache.flink.runtime.io.network.buffer;
 
 import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.core.memory.MemoryType;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Queue;
 import java.util.Set;
 import java.util.concurrent.ArrayBlockingQueue;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * The NetworkBufferPool is a fixed size pool of {@link MemorySegment} instances
  * for the network stack.
@@ -62,7 +67,9 @@ public class NetworkBufferPool implements BufferPoolFactory {
 	/**
 	 * Allocates all {@link MemorySegment} instances managed by this pool.
 	 */
-	public NetworkBufferPool(int numberOfSegmentsToAllocate, int segmentSize) {
+	public NetworkBufferPool(int numberOfSegmentsToAllocate, int segmentSize, MemoryType memoryType) {
+		checkNotNull(memoryType);
+		
 		this.totalNumberOfMemorySegments = numberOfSegmentsToAllocate;
 		this.memorySegmentSize = segmentSize;
 
@@ -72,12 +79,25 @@ public class NetworkBufferPool implements BufferPoolFactory {
 			this.availableMemorySegments = new ArrayBlockingQueue<MemorySegment>(numberOfSegmentsToAllocate);
 		}
 		catch (OutOfMemoryError err) {
-			throw new OutOfMemoryError("Could not allocate buffer queue of length " + numberOfSegmentsToAllocate);
+			throw new OutOfMemoryError("Could not allocate buffer queue of length "
+					+ numberOfSegmentsToAllocate + " - " + err.getMessage());
 		}
 
 		try {
-			for (int i = 0; i < numberOfSegmentsToAllocate; i++) {
-				availableMemorySegments.add(new MemorySegment(new byte[segmentSize]));
+			if (memoryType == MemoryType.HEAP) {
+				for (int i = 0; i < numberOfSegmentsToAllocate; i++) {
+					byte[] memory = new byte[segmentSize];
+					availableMemorySegments.add(MemorySegmentFactory.wrapPooledHeapMemory(memory, null));
+				}
+			}
+			else if (memoryType == MemoryType.OFF_HEAP) {
+				for (int i = 0; i < numberOfSegmentsToAllocate; i++) {
+					ByteBuffer memory = ByteBuffer.allocateDirect(segmentSize);
+					availableMemorySegments.add(MemorySegmentFactory.wrapPooledOffHeapMemory(memory, null));
+				}
+			}
+			else {
+				throw new IllegalArgumentException("Unknown memory type " + memoryType);
 			}
 		}
 		catch (OutOfMemoryError err) {
@@ -93,7 +113,7 @@ public class NetworkBufferPool implements BufferPoolFactory {
 			throw new OutOfMemoryError("Could not allocate enough memory segments for NetworkBufferPool " +
 					"(required (Mb): " + requiredMb +
 					", allocated (Mb): " + allocatedMb +
-					", missing (Mb): " + missingMb + ").");
+					", missing (Mb): " + missingMb + "). Cause: " + err.getMessage());
 		}
 
 		long allocatedMb = (sizeInLong * availableMemorySegments.size()) >> 20;

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
index 3b7d921..ee015c2 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/PartitionRequestClientHandler.java
@@ -23,6 +23,7 @@ import io.netty.channel.ChannelHandlerContext;
 import io.netty.channel.ChannelInboundHandlerAdapter;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.network.buffer.Buffer;
 import org.apache.flink.runtime.io.network.buffer.BufferProvider;
 import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler;
@@ -301,7 +302,8 @@ class PartitionRequestClientHandler extends ChannelInboundHandlerAdapter {
 				byte[] byteArray = new byte[bufferOrEvent.getSize()];
 				bufferOrEvent.getNettyBuffer().readBytes(byteArray);
 
-				Buffer buffer = new Buffer(new MemorySegment(byteArray), FreeingBufferRecycler.INSTANCE, false);
+				MemorySegment memSeg = MemorySegmentFactory.wrap(byteArray);
+				Buffer buffer = new Buffer(memSeg, FreeingBufferRecycler.INSTANCE, false);
 
 				inputChannel.onBuffer(buffer, bufferOrEvent.sequenceNumber);
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
index 5b91668..2c0c744 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/SpilledSubpartitionViewSyncIO.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.io.network.partition;
 
 import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
 import org.apache.flink.runtime.io.disk.iomanager.BufferFileReader;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.SynchronousBufferFileReader;
@@ -136,7 +137,7 @@ class SpilledSubpartitionViewSyncIO implements ResultSubpartitionView {
 
 			synchronized (buffers) {
 				for (int i = 0; i < numberOfBuffers; i++) {
-					buffers.add(new Buffer(new MemorySegment(new byte[memorySegmentSize]), this));
+					buffers.add(new Buffer(MemorySegmentFactory.allocateUnpooledSegment(memorySegmentSize), this));
 				}
 			}
 		}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
index ca1ade9..7776894 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/iterative/io/SerializedUpdateBuffer.java
@@ -33,8 +33,8 @@ import org.apache.flink.runtime.io.disk.iomanager.BlockChannelReader;
 import org.apache.flink.runtime.io.disk.iomanager.BlockChannelWriter;
 import org.apache.flink.runtime.io.disk.iomanager.FileIOChannel;
 import org.apache.flink.runtime.io.disk.iomanager.IOManager;
-import org.apache.flink.runtime.memorymanager.AbstractPagedInputView;
-import org.apache.flink.runtime.memorymanager.AbstractPagedOutputView;
+import org.apache.flink.runtime.memory.AbstractPagedInputView;
+import org.apache.flink.runtime.memory.AbstractPagedOutputView;
 
 public class SerializedUpdateBuffer extends AbstractPagedOutputView {
 

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java
new file mode 100644
index 0000000..788eebb
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedInputView.java
@@ -0,0 +1,568 @@
+/*
+ * 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.runtime.memory;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+
+/**
+ * The base class for all input views that are backed by multiple memory pages. This base class contains all
+ * decoding methods to read data from a page and detect page boundary crossing. The concrete sub classes must
+ * implement the methods to provide the next memory page once the boundary is crossed.
+ */
+public abstract class AbstractPagedInputView implements DataInputView {
+	
+	private MemorySegment currentSegment;
+	
+	protected final int headerLength;				// the number of bytes to skip at the beginning of each segment
+	
+	private int positionInSegment;					// the offset in the current segment
+	
+	private int limitInSegment;						// the limit in the current segment before switching to the next
+	
+	private byte[] utfByteBuffer;					// reusable byte buffer for utf-8 decoding
+	private char[] utfCharBuffer;					// reusable char buffer for utf-8 decoding
+	
+	
+	// --------------------------------------------------------------------------------------------
+	//                                    Constructors
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Creates a new view that starts with the given segment. The input starts directly after the header
+	 * of the given page. If the header size is zero, it starts at the beginning. The specified initial
+	 * limit describes up to which position data may be read from the current segment, before the view must
+	 * advance to the next segment.
+	 * 
+	 * @param initialSegment The memory segment to start reading from.
+	 * @param initialLimit The position one after the last valid byte in the initial segment.
+	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header. This
+	 *                     length must be the same for all memory segments.
+	 */
+	protected AbstractPagedInputView(MemorySegment initialSegment, int initialLimit, int headerLength) {
+		this.headerLength = headerLength;
+		this.positionInSegment = headerLength;
+		seekInput(initialSegment, headerLength, initialLimit);
+	}
+	
+	/**
+	 * Creates a new view that is initially not bound to a memory segment. This constructor is typically
+	 * for views that always seek first.
+	 * <p>
+	 * WARNING: The view is not readable until the first call to either {@link #advance()}, 
+	 * or to {@link #seekInput(MemorySegment, int, int)}.
+	 * 
+	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
+	 */
+	protected AbstractPagedInputView(int headerLength) {
+		this.headerLength = headerLength;
+	}
+
+	// --------------------------------------------------------------------------------------------
+	//                                  Page Management
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Gets the memory segment that will be used to read the next bytes from. If the segment is exactly exhausted,
+	 * meaning that the last byte read was the last byte available in the segment, then this segment will
+	 * not serve the next bytes. The segment to serve the next bytes will be obtained through the
+	 * {@link #nextSegment(MemorySegment)} method.
+	 * 
+	 * @return The current memory segment.
+	 */
+	public MemorySegment getCurrentSegment() {
+		return this.currentSegment;
+	}
+	
+	/**
+	 * Gets the position from which the next byte will be read. If that position is equal to the current limit,
+	 * then the next byte will be read from next segment.
+	 * 
+	 * @return The position from which the next byte will be read.
+	 * @see #getCurrentSegmentLimit()
+	 */
+	public int getCurrentPositionInSegment() {
+		return this.positionInSegment;
+	}
+	
+	/**
+	 * Gets the current limit in the memory segment. This value points to the byte one after the last valid byte
+	 * in the memory segment.
+	 * 
+	 * @return The current limit in the memory segment.
+	 * @see #getCurrentPositionInSegment()
+	 */
+	public int getCurrentSegmentLimit() {
+		return this.limitInSegment;
+	}
+	
+	/**
+	 * The method by which concrete subclasses realize page crossing. This method is invoked when the current page
+	 * is exhausted and a new page is required to continue the reading. If no further page is available, this
+	 * method must throw an {@link EOFException}.
+	 *  
+	 * @param current The current page that was read to its limit. May be {@code null}, if this method is
+	 *                invoked for the first time.
+	 * @return The next page from which the reading should continue. May not be {@code null}. If the input is
+	 *         exhausted, an {@link EOFException} must be thrown instead.
+	 *         
+	 * @throws EOFException Thrown, if no further segment is available.
+	 * @throws IOException Thrown, if the method cannot provide the next page due to an I/O related problem.
+	 */
+	protected abstract MemorySegment nextSegment(MemorySegment current) throws EOFException, IOException;
+	
+	/**
+	 * Gets the limit for reading bytes from the given memory segment. This method must return the position
+	 * of the byte after the last valid byte in the given memory segment. When the position returned by this
+	 * method is reached, the view will attempt to switch to the next memory segment.
+	 * 
+	 * @param segment The segment to determine the limit for.
+	 * @return The limit for the given memory segment.
+	 */
+	protected abstract int getLimitForSegment(MemorySegment segment);
+	
+	/**
+	 * Advances the view to the next memory segment. The reading will continue after the header of the next
+	 * segment. This method uses {@link #nextSegment(MemorySegment)} and {@link #getLimitForSegment(MemorySegment)}
+	 * to get the next segment and set its limit.
+	 * 
+	 * @throws IOException Thrown, if the next segment could not be obtained.
+	 * 
+	 * @see #nextSegment(MemorySegment)
+	 * @see #getLimitForSegment(MemorySegment)
+	 */
+	protected final void advance() throws IOException {
+		// note: this code ensures that in case of EOF, we stay at the same position such that
+		// EOF is reproducible (if nextSegment throws a reproducible EOFException)
+		this.currentSegment = nextSegment(this.currentSegment);
+		this.limitInSegment = getLimitForSegment(this.currentSegment);
+		this.positionInSegment = this.headerLength;
+	}
+	
+	/**
+	 * Sets the internal state of the view such that the next bytes will be read from the given memory segment,
+	 * starting at the given position. The memory segment will provide bytes up to the given limit position.
+	 * 
+	 * @param segment The segment to read the next bytes from.
+	 * @param positionInSegment The position in the segment to start reading from.
+	 * @param limitInSegment The limit in the segment. When reached, the view will attempt to switch to
+	 *                       the next segment.
+	 */
+	protected void seekInput(MemorySegment segment, int positionInSegment, int limitInSegment) {
+		this.currentSegment = segment;
+		this.positionInSegment = positionInSegment;
+		this.limitInSegment = limitInSegment;
+	}
+	
+	/**
+	 * Clears the internal state of the view. After this call, all read attempts will fail, until the
+	 * {@link #advance()} or {@link #seekInput(MemorySegment, int, int)} method have been invoked.
+	 */
+	protected void clear() {
+		this.currentSegment = null;
+		this.positionInSegment = this.headerLength;
+		this.limitInSegment = headerLength;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                               Data Input Specific methods
+	// --------------------------------------------------------------------------------------------
+
+	@Override
+	public int read(byte[] b) throws IOException{
+		return read(b,0,b.length);
+	}
+
+	@Override
+	public int read(byte[] b, int off, int len) throws IOException{
+		if (off < 0 || len < 0 || off + len > b.length) {
+			throw new IndexOutOfBoundsException();
+		}
+
+		int remaining = this.limitInSegment - this.positionInSegment;
+		if (remaining >= len) {
+			this.currentSegment.get(this.positionInSegment, b, off, len);
+			this.positionInSegment += len;
+			return len;
+		}
+		else {
+			if (remaining == 0) {
+				try {
+					advance();
+				}
+				catch (EOFException eof) {
+					return -1;
+				}
+				remaining = this.limitInSegment - this.positionInSegment;
+			}
+
+			int bytesRead = 0;
+			while (true) {
+				int toRead = Math.min(remaining, len-bytesRead);
+				this.currentSegment.get(this.positionInSegment, b, off, toRead);
+				off += toRead;
+				bytesRead += toRead;
+
+				if (len > bytesRead) {
+					try {
+						advance();
+					}
+					catch (EOFException eof) {
+						this.positionInSegment += toRead;
+						return bytesRead;
+					}
+					remaining = this.limitInSegment - this.positionInSegment;
+				}
+				else {
+					this.positionInSegment += toRead;
+					break;
+				}
+			}
+			return len;
+		}
+	}
+	
+	@Override
+	public void readFully(byte[] b) throws IOException {
+		readFully(b, 0, b.length);
+	}
+
+	@Override
+	public void readFully(byte[] b, int off, int len) throws IOException {
+		int bytesRead = read(b,off,len);
+
+		if(bytesRead < len){
+			throw new EOFException("There is no enough data left in the DataInputView.");
+		}
+	}
+
+	@Override
+	public boolean readBoolean() throws IOException {
+		return readByte() == 1;
+	}
+
+	@Override
+	public byte readByte() throws IOException {
+		if (this.positionInSegment < this.limitInSegment) {
+			return this.currentSegment.get(this.positionInSegment++);
+		}
+		else {
+			advance();
+			return readByte();
+		}
+	}
+
+	@Override
+	public int readUnsignedByte() throws IOException {
+		return readByte() & 0xff;
+	}
+
+	@Override
+	public short readShort() throws IOException {
+		if (this.positionInSegment < this.limitInSegment - 1) {
+			final short v = this.currentSegment.getShortBigEndian(this.positionInSegment);
+			this.positionInSegment += 2;
+			return v;
+		}
+		else if (this.positionInSegment == this.limitInSegment) {
+			advance();
+			return readShort();
+		}
+		else {
+			return (short) ((readUnsignedByte() << 8) | readUnsignedByte());
+		}
+	}
+
+	@Override
+	public int readUnsignedShort() throws IOException {
+		if (this.positionInSegment < this.limitInSegment - 1) {
+			final int v = this.currentSegment.getShortBigEndian(this.positionInSegment) & 0xffff;
+			this.positionInSegment += 2;
+			return v;
+		}
+		else if (this.positionInSegment == this.limitInSegment) {
+			advance();
+			return readUnsignedShort();
+		}
+		else {
+			return (readUnsignedByte() << 8) | readUnsignedByte();
+		}
+	}
+
+	@Override
+	public char readChar() throws IOException  {
+		if (this.positionInSegment < this.limitInSegment - 1) {
+			final char v = this.currentSegment.getCharBigEndian(this.positionInSegment);
+			this.positionInSegment += 2;
+			return v;
+		}
+		else if (this.positionInSegment == this.limitInSegment) {
+			advance();
+			return readChar();
+		}
+		else {
+			return (char) ((readUnsignedByte() << 8) | readUnsignedByte());
+		}
+	}
+
+	@Override
+	public int readInt() throws IOException {
+		if (this.positionInSegment < this.limitInSegment - 3) {
+			final int v = this.currentSegment.getIntBigEndian(this.positionInSegment);
+			this.positionInSegment += 4;
+			return v;
+		}
+		else if (this.positionInSegment == this.limitInSegment) {
+			advance();
+			return readInt();
+		}
+		else {
+			return (readUnsignedByte() << 24) |
+				(readUnsignedByte() << 16) |
+				(readUnsignedByte() <<  8) |
+					readUnsignedByte();
+		}
+	}
+
+	@Override
+	public long readLong() throws IOException {
+		if (this.positionInSegment < this.limitInSegment - 7) {
+			final long v = this.currentSegment.getLongBigEndian(this.positionInSegment);
+			this.positionInSegment += 8;
+			return v;
+		}
+		else if (this.positionInSegment == this.limitInSegment) {
+			advance();
+			return readLong();
+		}
+		else {
+			long l = 0L;
+			l |= ((long) readUnsignedByte()) << 56;
+			l |= ((long) readUnsignedByte()) << 48;
+			l |= ((long) readUnsignedByte()) << 40;
+			l |= ((long) readUnsignedByte()) << 32;
+			l |= ((long) readUnsignedByte()) << 24;
+			l |= ((long) readUnsignedByte()) << 16;
+			l |= ((long) readUnsignedByte()) <<  8;
+			l |= (long) readUnsignedByte();
+			return l;
+		}
+	}
+
+	@Override
+	public float readFloat() throws IOException {
+		return Float.intBitsToFloat(readInt());
+	}
+
+	@Override
+	public double readDouble() throws IOException {
+		return Double.longBitsToDouble(readLong());
+	}
+
+	@Override
+	public String readLine() throws IOException {
+		final StringBuilder bld = new StringBuilder(32);
+		
+		try {
+			int b;
+			while ((b = readUnsignedByte()) != '\n') {
+				if (b != '\r') {
+					bld.append((char) b);
+				}
+			}
+		}
+		catch (EOFException eofex) {}
+
+		if (bld.length() == 0) {
+			return null;
+		}
+		
+		// trim a trailing carriage return
+		int len = bld.length();
+		if (len > 0 && bld.charAt(len - 1) == '\r') {
+			bld.setLength(len - 1);
+		}
+		return bld.toString();
+	}
+
+	@Override
+	public String readUTF() throws IOException {
+		final int utflen = readUnsignedShort();
+		
+		final byte[] bytearr;
+		final char[] chararr;
+		
+		if (this.utfByteBuffer == null || this.utfByteBuffer.length < utflen) {
+			bytearr = new byte[utflen];
+			this.utfByteBuffer = bytearr;
+		} else {
+			bytearr = this.utfByteBuffer;
+		}
+		if (this.utfCharBuffer == null || this.utfCharBuffer.length < utflen) {
+			chararr = new char[utflen];
+			this.utfCharBuffer = chararr;
+		} else {
+			chararr = this.utfCharBuffer;
+		}
+
+		int c, char2, char3;
+		int count = 0;
+		int chararr_count = 0;
+
+		readFully(bytearr, 0, utflen);
+
+		while (count < utflen) {
+			c = (int) bytearr[count] & 0xff;
+			if (c > 127) {
+				break;
+			}
+			count++;
+			chararr[chararr_count++] = (char) c;
+		}
+
+		while (count < utflen) {
+			c = (int) bytearr[count] & 0xff;
+			switch (c >> 4) {
+			case 0:
+			case 1:
+			case 2:
+			case 3:
+			case 4:
+			case 5:
+			case 6:
+			case 7:
+				/* 0xxxxxxx */
+				count++;
+				chararr[chararr_count++] = (char) c;
+				break;
+			case 12:
+			case 13:
+				/* 110x xxxx 10xx xxxx */
+				count += 2;
+				if (count > utflen) {
+					throw new UTFDataFormatException("malformed input: partial character at end");
+				}
+				char2 = (int) bytearr[count - 1];
+				if ((char2 & 0xC0) != 0x80) {
+					throw new UTFDataFormatException("malformed input around byte " + count);
+				}
+				chararr[chararr_count++] = (char) (((c & 0x1F) << 6) | (char2 & 0x3F));
+				break;
+			case 14:
+				/* 1110 xxxx 10xx xxxx 10xx xxxx */
+				count += 3;
+				if (count > utflen) {
+					throw new UTFDataFormatException("malformed input: partial character at end");
+				}
+				char2 = (int) bytearr[count - 2];
+				char3 = (int) bytearr[count - 1];
+				if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) {
+					throw new UTFDataFormatException("malformed input around byte " + (count - 1));
+				}
+				chararr[chararr_count++] = (char) (((c & 0x0F) << 12) | ((char2 & 0x3F) << 6) | ((char3 & 0x3F) << 0));
+				break;
+			default:
+				/* 10xx xxxx, 1111 xxxx */
+				throw new UTFDataFormatException("malformed input around byte " + count);
+			}
+		}
+		// The number of chars produced may be less than utflen
+		return new String(chararr, 0, chararr_count);
+	}
+	
+	@Override
+	public int skipBytes(int n) throws IOException {
+		if (n < 0) {
+			throw new IllegalArgumentException();
+		}
+		
+		int remaining = this.limitInSegment - this.positionInSegment;
+		if (remaining >= n) {
+			this.positionInSegment += n;
+			return n;
+		}
+		else {
+			if (remaining == 0) {
+				try {
+					advance();
+				} catch (EOFException eofex) {
+					return 0;
+				}
+				remaining = this.limitInSegment - this.positionInSegment;
+			}
+			
+			int skipped = 0;
+			while (true) {
+				int toSkip = Math.min(remaining, n);
+				n -= toSkip;
+				skipped += toSkip;
+				
+				if (n > 0) {
+					try {
+						advance();
+					} catch (EOFException eofex) {
+						return skipped;
+					}
+					remaining = this.limitInSegment - this.positionInSegment;	
+				}
+				else {
+					this.positionInSegment += toSkip;
+					break;
+				}
+			}
+			return skipped;
+		}
+	}
+
+	@Override
+	public void skipBytesToRead(int numBytes) throws IOException {
+		if (numBytes < 0) {
+			throw new IllegalArgumentException();
+		}
+		
+		int remaining = this.limitInSegment - this.positionInSegment;
+		if (remaining >= numBytes) {
+			this.positionInSegment += numBytes;
+		}
+		else {
+			if (remaining == 0) {
+				advance();
+				remaining = this.limitInSegment - this.positionInSegment;
+			}
+			
+			while (true) {
+				if (numBytes > remaining) {
+					numBytes -= remaining;
+					advance();
+					remaining = this.limitInSegment - this.positionInSegment;	
+				}
+				else {
+					this.positionInSegment += numBytes;
+					break;
+				}
+			}
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java
new file mode 100644
index 0000000..482d82e
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/AbstractPagedOutputView.java
@@ -0,0 +1,414 @@
+/*
+ * 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.runtime.memory;
+
+import java.io.IOException;
+import java.io.UTFDataFormatException;
+
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.MemorySegment;
+
+
+/**
+ * The base class for all output views that are backed by multiple memory pages. This base class contains all
+ * encoding methods to write data to a page and detect page boundary crossing. The concrete sub classes must
+ * implement the methods to collect the current page and provide the next memory page once the boundary is crossed.
+ * <p>
+ * The paging assumes that all memory segments are of the same size.
+ */
+public abstract class AbstractPagedOutputView implements DataOutputView {
+	
+	private MemorySegment currentSegment;			// the current memory segment to write to
+	
+	protected final int segmentSize;				// the size of the memory segments
+	
+	protected final int headerLength;				// the number of bytes to skip at the beginning of each segment
+	
+	private int positionInSegment;					// the offset in the current segment
+	
+	private byte[] utfBuffer;						// the reusable array for UTF encodings
+	
+	
+	// --------------------------------------------------------------------------------------------
+	//                                    Constructors
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * Creates a new output view that writes initially to the given initial segment. All segments in the
+	 * view have to be of the given {@code segmentSize}. A header of length {@code headerLength} is left
+	 * at the beginning of each segment.
+	 * 
+	 * @param initialSegment The segment that the view starts writing to.
+	 * @param segmentSize The size of the memory segments.
+	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
+	 */
+	protected AbstractPagedOutputView(MemorySegment initialSegment, int segmentSize, int headerLength) {
+		if (initialSegment == null) {
+			throw new NullPointerException("Initial Segment may not be null");
+		}
+		this.segmentSize = segmentSize;
+		this.headerLength = headerLength;
+		this.currentSegment = initialSegment;
+		this.positionInSegment = headerLength;
+	}
+	
+	/**
+	 * @param segmentSize The size of the memory segments.
+	 * @param headerLength The number of bytes to skip at the beginning of each segment for the header.
+	 */
+	protected AbstractPagedOutputView(int segmentSize, int headerLength)
+	{
+		this.segmentSize = segmentSize;
+		this.headerLength = headerLength;
+	}
+	
+
+	// --------------------------------------------------------------------------------------------
+	//                                  Page Management
+	// --------------------------------------------------------------------------------------------
+	
+	/**
+	 * 
+	 * This method must return a segment. If no more segments are available, it must throw an
+	 * {@link java.io.EOFException}.
+	 * 
+	 * @param current The current memory segment
+	 * @param positionInCurrent The position in the segment, one after the last valid byte.
+	 * @return The next memory segment. 
+	 * 
+	 * @throws IOException
+	 */
+	protected abstract MemorySegment nextSegment(MemorySegment current, int positionInCurrent) throws IOException;
+	
+	
+	/**
+	 * Gets the segment that the view currently writes to.
+	 * 
+	 * @return The segment the view currently writes to.
+	 */
+	public MemorySegment getCurrentSegment() {
+		return this.currentSegment;
+	}
+	
+	/**
+	 * Gets the current write position (the position where the next bytes will be written)
+	 * in the current memory segment.
+	 * 
+	 * @return The current write offset in the current memory segment.
+	 */
+	public int getCurrentPositionInSegment() {
+		return this.positionInSegment;
+	}
+	
+	/**
+	 * Gets the size of the segments used by this view.
+	 * 
+	 * @return The memory segment size.
+	 */
+	public int getSegmentSize() {
+		return this.segmentSize;
+	}
+	
+	/**
+	 * Moves the output view to the next page. This method invokes internally the
+	 * {@link #nextSegment(MemorySegment, int)} method to give the current memory segment to the concrete subclass' 
+	 * implementation and obtain the next segment to write to. Writing will continue inside the new segment
+	 * after the header.
+	 * 
+	 * @throws IOException Thrown, if the current segment could not be processed or a new segment could not
+	 *                     be obtained. 
+	 */
+	protected void advance() throws IOException {
+		this.currentSegment = nextSegment(this.currentSegment, this.positionInSegment);
+		this.positionInSegment = this.headerLength;
+	}
+	
+	/**
+	 * Sets the internal state to the given memory segment and the given position within the segment. 
+	 * 
+	 * @param seg The memory segment to write the next bytes to.
+	 * @param position The position to start writing the next bytes to.
+	 */
+	protected void seekOutput(MemorySegment seg, int position) {
+		this.currentSegment = seg;
+		this.positionInSegment = position;
+	}
+	
+	/**
+	 * Clears the internal state. Any successive write calls will fail until either {@link #advance()} or
+	 * {@link #seekOutput(MemorySegment, int)} is called. 
+	 * 
+	 * @see #advance()
+	 * @see #seekOutput(MemorySegment, int)
+	 */
+	protected void clear() {
+		this.currentSegment = null;
+		this.positionInSegment = this.headerLength;
+	}
+	
+	// --------------------------------------------------------------------------------------------
+	//                               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;
+		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-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java
new file mode 100644
index 0000000..ebeb47d
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/ListMemorySegmentSource.java
@@ -0,0 +1,48 @@
+/*
+ * 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.runtime.memory;
+
+import java.util.List;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentSource;
+
+/**
+ * Simple memory segment source that draws segments from a list.
+ * 
+ */
+public class ListMemorySegmentSource implements MemorySegmentSource
+{
+	private final List<MemorySegment> segments;
+	
+	public ListMemorySegmentSource(final List<MemorySegment> memorySegments) {
+		this.segments = memorySegments;
+	}
+	
+
+	@Override
+	public MemorySegment nextSegment() {
+		if (this.segments.size() > 0) {
+			return this.segments.remove(this.segments.size() - 1);
+		} else {
+			return null;
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java
new file mode 100644
index 0000000..112d6f7
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryAllocationException.java
@@ -0,0 +1,44 @@
+/*
+ * 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.runtime.memory;
+
+/**
+ * An exception to be thrown when a memory allocation operation is not successful.
+ */
+public class MemoryAllocationException extends Exception {
+	
+	private static final long serialVersionUID = -403983866457947012L;
+
+	public MemoryAllocationException() {
+		super();
+	}
+
+	public MemoryAllocationException(String message, Throwable cause) {
+		super(message, cause);
+	}
+
+	public MemoryAllocationException(String message) {
+		super(message);
+	}
+
+	public MemoryAllocationException(Throwable cause) {
+		super(cause);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/655a891d/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java
new file mode 100644
index 0000000..094d065
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/memory/MemoryManager.java
@@ -0,0 +1,700 @@
+/*
+ * 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.runtime.memory;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.ConcurrentModificationException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
+import org.apache.flink.runtime.util.MathUtils;
+
+/**
+ * The memory manager governs the memory that Flink uses for sorting, hashing, and caching. Memory
+ * is represented in segments of equal size. Operators allocate the memory by requesting a number
+ * of memory segments.
+ * <p>
+ * The memory may be represented as on-heap byte arrays ({@link HeapMemorySegment}), or as off-heap
+ * memory regions ({@link HybridMemorySegment}). Which kind of memory the MemoryManager serves can
+ * be passed as an argument to the initialization.
+ * <p>
+ * The memory manager can either pre-allocate all memory, or allocate the memory on demand. In the
+ * former version, memory will be occupied and reserved from start on, which means that no OutOfMemoryError
+ * can come while requesting memory. Released memory will also return to the MemoryManager's pool.
+ * On-demand allocation means that the memory manager only keeps track how many memory segments are
+ * currently allocated (bookkeeping only). Releasing a memory segment will not add it back to the pool,
+ * but make it re-claimable by the garbage collector.
+ */
+public class MemoryManager {
+
+	/** The default memory page size. Currently set to 32 KiBytes. */
+	public static final int DEFAULT_PAGE_SIZE = 32 * 1024;
+
+	/** The minimal memory page size. Currently set to 4 KiBytes. */
+	public static final int MIN_PAGE_SIZE = 4 * 1024;
+
+	// ------------------------------------------------------------------------
+
+	/** The lock used on the shared structures. */
+	private final Object lock = new Object();
+
+	/** The memory pool from which we draw memory segments. Specific to on-heap or off-heap memory */
+	private final MemoryPool memoryPool;
+	
+	/** Memory segments allocated per memory owner */
+	private final HashMap<Object, Set<MemorySegment>> allocatedSegments;
+
+	/** The type of memory governed by this memory manager */
+	private final MemoryType memoryType;
+	
+	/** mask used to round down sizes to multiples of the page size */
+	private final long roundingMask;
+
+	/** The size of the memory segments */
+	private final int pageSize;
+
+	/** The initial total size, for verification. */
+	private final int totalNumPages;
+
+	/** The total size of the memory managed by this memory manager */
+	private final long memorySize;
+
+	/** Number of slots of the task manager */
+	private final int numberOfSlots;
+
+	/** Flag marking whether the memory manager immediately allocates the memory */
+	private final boolean isPreAllocated;
+
+	/** The number of memory pages that have not been allocated and are available for lazy allocation */
+	private int numNonAllocatedPages;
+
+	/** flag whether the close() has already been invoked */
+	private boolean isShutDown;
+
+
+	/**
+	 * Creates a memory manager with the given capacity, using the default page size.
+	 *
+	 * @param memorySize The total size of the memory to be managed by this memory manager.
+	 * @param numberOfSlots The number of slots of the task manager.
+	 */
+	public MemoryManager(long memorySize, int numberOfSlots) {
+		this(memorySize, numberOfSlots, DEFAULT_PAGE_SIZE, MemoryType.HEAP, true);
+	}
+
+	/**
+	 * Creates a memory manager with the given capacity and given page size.
+	 *
+	 * @param memorySize The total size of the memory to be managed by this memory manager.
+	 * @param numberOfSlots The number of slots of the task manager.
+	 * @param pageSize The size of the pages handed out by the memory manager.
+	 * @param memoryType The type of memory (heap / off-heap) that the memory manager should allocate.
+	 * @param preAllocateMemory True, if the memory manager should immediately allocate all memory, false
+	 *                          if it should allocate and release the memory as needed.
+	 */
+	public MemoryManager(long memorySize, int numberOfSlots, int pageSize,
+							MemoryType memoryType, boolean preAllocateMemory) {
+		// sanity checks
+		if (memoryType == null) {
+			throw new NullPointerException();
+		}
+		if (memorySize <= 0) {
+			throw new IllegalArgumentException("Size of total memory must be positive.");
+		}
+		if (pageSize < MIN_PAGE_SIZE) {
+			throw new IllegalArgumentException("The page size must be at least " + MIN_PAGE_SIZE + " bytes.");
+		}
+		if (!MathUtils.isPowerOf2(pageSize)) {
+			throw new IllegalArgumentException("The given page size is not a power of two.");
+		}
+
+		this.memoryType = memoryType;
+		this.memorySize = memorySize;
+		this.numberOfSlots = numberOfSlots;
+
+		// assign page size and bit utilities
+		this.pageSize = pageSize;
+		this.roundingMask = ~((long) (pageSize - 1));
+
+		final long numPagesLong = memorySize / pageSize;
+		if (numPagesLong > Integer.MAX_VALUE) {
+			throw new IllegalArgumentException("The given number of memory bytes (" + memorySize
+					+ ") corresponds to more than MAX_INT pages.");
+		}
+		this.totalNumPages = (int) numPagesLong;
+		if (this.totalNumPages < 1) {
+			throw new IllegalArgumentException("The given amount of memory amounted to less than one page.");
+		}
+
+		this.allocatedSegments = new HashMap<Object, Set<MemorySegment>>();
+		this.isPreAllocated = preAllocateMemory;
+
+		this.numNonAllocatedPages = preAllocateMemory ? 0 : this.totalNumPages;
+		final int memToAllocate = preAllocateMemory ? this.totalNumPages : 0;
+		
+		switch (memoryType) {
+			case HEAP:
+				this.memoryPool = new HeapMemoryPool(memToAllocate, pageSize);
+				break;
+			case OFF_HEAP:
+				this.memoryPool = new HybridOffHeapMemoryPool(memToAllocate, pageSize);
+				break;
+			default:
+				throw new IllegalArgumentException("unrecognized memory type: " + memoryType);
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Shutdown
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Shuts the memory manager down, trying to release all the memory it managed. Depending
+	 * on implementation details, the memory does not necessarily become reclaimable by the
+	 * garbage collector, because there might still be references to allocated segments in the
+	 * code that allocated them from the memory manager.
+	 */
+	public void shutdown() {
+		// -------------------- BEGIN CRITICAL SECTION -------------------
+		synchronized (lock)
+		{
+			if (!isShutDown) {
+				// mark as shutdown and release memory
+				isShutDown = true;
+				numNonAllocatedPages = 0;
+
+				// go over all allocated segments and release them
+				for (Set<MemorySegment> segments : allocatedSegments.values()) {
+					for (MemorySegment seg : segments) {
+						seg.free();
+					}
+				}
+				
+				memoryPool.clear();
+			}
+		}
+		// -------------------- END CRITICAL SECTION -------------------
+	}
+
+	/**
+	 * Checks whether the MemoryManager has been shut down.
+	 *
+	 * @return True, if the memory manager is shut down, false otherwise.
+	 */
+	public boolean isShutdown() {
+		return isShutDown;
+	}
+
+	/**
+	 * Checks if the memory manager all memory available.
+	 *
+	 * @return True, if the memory manager is empty and valid, false if it is not empty or corrupted.
+	 */
+	public boolean verifyEmpty() {
+		synchronized (lock) {
+			return isPreAllocated ?
+					memoryPool.getNumberOfAvailableMemorySegments() == totalNumPages :
+					numNonAllocatedPages == totalNumPages;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+	//  Memory allocation and release
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Allocates a set of memory segments from this memory manager. If the memory manager pre-allocated the
+	 * segments, they will be taken from the pool of memory segments. Otherwise, they will be allocated
+	 * as part of this call.
+	 *
+	 * @param owner The owner to associate with the memory segment, for the fallback release.
+	 * @param numPages The number of pages to allocate.
+	 * @return A list with the memory segments.   
+	 * @throws MemoryAllocationException Thrown, if this memory manager does not have the requested amount
+	 *                                   of memory pages any more.
+	 */
+	public List<MemorySegment> allocatePages(Object owner, int numPages) throws MemoryAllocationException {
+		final ArrayList<MemorySegment> segs = new ArrayList<MemorySegment>(numPages);
+		allocatePages(owner, segs, numPages);
+		return segs;
+	}
+
+	/**
+	 * Allocates a set of memory segments from this memory manager. If the memory manager pre-allocated the
+	 * segments, they will be taken from the pool of memory segments. Otherwise, they will be allocated
+	 * as part of this call.
+	 * 
+	 * @param owner The owner to associate with the memory segment, for the fallback release.
+	 * @param target The list into which to put the allocated memory pages.
+	 * @param numPages The number of pages to allocate.
+	 * @throws MemoryAllocationException Thrown, if this memory manager does not have the requested amount
+	 *                                   of memory pages any more.
+	 */
+	public void allocatePages(Object owner, List<MemorySegment> target, int numPages)
+			throws MemoryAllocationException
+	{
+		// sanity check
+		if (owner == null) {
+			throw new IllegalArgumentException("The memory owner must not be null.");
+		}
+
+		// reserve array space, if applicable
+		if (target instanceof ArrayList) {
+			((ArrayList<MemorySegment>) target).ensureCapacity(numPages);
+		}
+
+		// -------------------- BEGIN CRITICAL SECTION -------------------
+		synchronized (lock)
+		{
+			if (isShutDown) {
+				throw new IllegalStateException("Memory manager has been shut down.");
+			}
+
+			// in the case of pre-allocated memory, the 'numNonAllocatedPages' is zero, in the
+			// lazy case, the 'freeSegments.size()' is zero.
+			if (numPages > (memoryPool.getNumberOfAvailableMemorySegments() + numNonAllocatedPages)) {
+				throw new MemoryAllocationException("Could not allocate " + numPages + " pages. Only " +
+						(memoryPool.getNumberOfAvailableMemorySegments() + numNonAllocatedPages)
+						+ " pages are remaining.");
+			}
+
+			Set<MemorySegment> segmentsForOwner = allocatedSegments.get(owner);
+			if (segmentsForOwner == null) {
+				segmentsForOwner = new HashSet<MemorySegment>(numPages);
+				allocatedSegments.put(owner, segmentsForOwner);
+			}
+
+			if (isPreAllocated) {
+				for (int i = numPages; i > 0; i--) {
+					MemorySegment segment = memoryPool.requestSegmentFromPool(owner);
+					target.add(segment);
+					segmentsForOwner.add(segment);
+				}
+			}
+			else {
+				for (int i = numPages; i > 0; i--) {
+					MemorySegment segment = memoryPool.allocateNewSegment(owner);
+					target.add(segment);
+					segmentsForOwner.add(segment);
+				}
+				numNonAllocatedPages -= numPages;
+			}
+		}
+		// -------------------- END CRITICAL SECTION -------------------
+	}
+
+	/**
+	 * Tries to release the memory for the specified segment. If the segment has already been released or
+	 * is null, the request is simply ignored.
+	 * <p>
+	 * If the memory manager manages pre-allocated memory, the memory segment goes back to the memory pool.
+	 * Otherwise, the segment is only freed and made eligible for reclamation by the GC.
+	 *
+	 * @param segment The segment to be released.
+	 * @throws IllegalArgumentException Thrown, if the given segment is of an incompatible type.
+	 */
+	public void release(MemorySegment segment) {
+		// check if segment is null or has already been freed
+		if (segment == null || segment.getOwner() == null) {
+			return;
+		}
+
+		final Object owner = segment.getOwner();
+		
+		// -------------------- BEGIN CRITICAL SECTION -------------------
+		synchronized (lock)
+		{
+			// prevent double return to this memory manager
+			if (segment.isFreed()) {
+				return;
+			}
+			if (isShutDown) {
+				throw new IllegalStateException("Memory manager has been shut down.");
+			}
+
+			// remove the reference in the map for the owner
+			try {
+				Set<MemorySegment> segsForOwner = this.allocatedSegments.get(owner);
+
+				if (segsForOwner != null) {
+					segsForOwner.remove(segment);
+					if (segsForOwner.isEmpty()) {
+						this.allocatedSegments.remove(owner);
+					}
+				}
+
+				if (isPreAllocated) {
+					// release the memory in any case
+					memoryPool.returnSegmentToPool(segment);
+				}
+				else {
+					segment.free();
+					numNonAllocatedPages++;
+				}
+			}
+			catch (Throwable t) {
+				throw new RuntimeException("Error removing book-keeping reference to allocated memory segment.", t);
+			}
+		}
+		// -------------------- END CRITICAL SECTION -------------------
+	}
+
+	/**
+	 * Tries to release many memory segments together.
+	 * <p>
+	 * If the memory manager manages pre-allocated memory, the memory segment goes back to the memory pool.
+	 * Otherwise, the segment is only freed and made eligible for reclamation by the GC.
+	 * 
+	 * @param segments The segments to be released.
+	 * @throws NullPointerException Thrown, if the given collection is null.
+	 * @throws IllegalArgumentException Thrown, id the segments are of an incompatible type.
+	 */
+	public void release(Collection<MemorySegment> segments) {
+		if (segments == null) {
+			return;
+		}
+
+		// -------------------- BEGIN CRITICAL SECTION -------------------
+		synchronized (lock)
+		{
+			if (isShutDown) {
+				throw new IllegalStateException("Memory manager has been shut down.");
+			}
+
+			// since concurrent modifications to the collection
+			// can disturb the release, we need to try potentially multiple times
+			boolean successfullyReleased = false;
+			do {
+				final Iterator<MemorySegment> segmentsIterator = segments.iterator();
+
+				Object lastOwner = null;
+				Set<MemorySegment> segsForOwner = null;
+
+				try {
+					// go over all segments
+					while (segmentsIterator.hasNext()) {
+
+						final MemorySegment seg = segmentsIterator.next();
+						if (seg == null || seg.isFreed()) {
+							continue;
+						}
+
+						final Object owner = seg.getOwner();
+
+						try {
+							// get the list of segments by this owner only if it is a different owner than for
+							// the previous one (or it is the first segment)
+							if (lastOwner != owner) {
+								lastOwner = owner;
+								segsForOwner = this.allocatedSegments.get(owner);
+							}
+
+							// remove the segment from the list
+							if (segsForOwner != null) {
+								segsForOwner.remove(seg);
+								if (segsForOwner.isEmpty()) {
+									this.allocatedSegments.remove(owner);
+								}
+							}
+
+							if (isPreAllocated) {
+								memoryPool.returnSegmentToPool(seg);
+							}
+							else {
+								seg.free();
+								numNonAllocatedPages++;
+							}
+						}
+						catch (Throwable t) {
+							throw new RuntimeException(
+									"Error removing book-keeping reference to allocated memory segment.", t);
+						}
+					}
+
+					segments.clear();
+
+					// the only way to exit the loop
+					successfullyReleased = true;
+				}
+				catch (ConcurrentModificationException e) {
+					// this may happen in the case where an asynchronous
+					// call releases the memory. fall through the loop and try again
+				}
+			} while (!successfullyReleased);
+		}
+		// -------------------- END CRITICAL SECTION -------------------
+	}
+
+	/**
+	 * Releases all memory segments for the given owner. 
+	 *
+	 * @param owner The owner memory segments are to be released.
+	 */
+	public void releaseAll(Object owner) {
+		if (owner == null) {
+			return;
+		}
+
+		// -------------------- BEGIN CRITICAL SECTION -------------------
+		synchronized (lock)
+		{
+			if (isShutDown) {
+				throw new IllegalStateException("Memory manager has been shut down.");
+			}
+
+			// get all segments
+			final Set<MemorySegment> segments = allocatedSegments.remove(owner);
+
+			// all segments may have been freed previously individually
+			if (segments == null || segments.isEmpty()) {
+				return;
+			}
+
+			// free each segment
+			if (isPreAllocated) {
+				for (MemorySegment seg : segments) {
+					memoryPool.returnSegmentToPool(seg);
+				}
+			}
+			else {
+				for (MemorySegment seg : segments) {
+					seg.free();
+				}
+				numNonAllocatedPages += segments.size();
+			}
+
+			segments.clear();
+		}
+		// -------------------- END CRITICAL SECTION -------------------
+	}
+
+	// ------------------------------------------------------------------------
+	//  Properties, sizes and size conversions
+	// ------------------------------------------------------------------------
+
+	/**
+	 * Gets the type of memory (heap / off-heap) managed by this memory manager.
+	 * 
+	 * @return The type of memory managed by this memory manager.
+	 */
+	public MemoryType getMemoryType() {
+		return memoryType;
+	}
+
+	/**
+	 * Checks whether this memory manager pre-allocates the memory.
+	 *
+	 * @return True if the memory manager pre-allocates the memory, false if it allocates as needed.
+	 */
+	public boolean isPreAllocated() {
+		return isPreAllocated;
+	}
+
+	/**
+	 * Gets the size of the pages handled by the memory manager.
+	 *
+	 * @return The size of the pages handled by the memory manager.
+	 */
+	public int getPageSize() {
+		return pageSize;
+	}
+
+	/**
+	 * Returns the total size of memory handled by this memory manager.
+	 *
+	 * @return The total size of memory.
+	 */
+	public long getMemorySize() {
+		return memorySize;
+	}
+
+	/**
+	 * Gets the total number of memory pages managed by this memory manager.
+	 *
+	 * @return The total number of memory pages managed by this memory manager.
+	 */
+	public int getTotalNumPages() {
+		return totalNumPages;
+	}
+
+	/**
+	 * Computes to how many pages the given number of bytes corresponds. If the given number of bytes is not an
+	 * exact multiple of a page size, the result is rounded down, such that a portion of the memory (smaller
+	 * than the page size) is not included.
+	 *
+	 * @param fraction the fraction of the total memory per slot
+	 * @return The number of pages to which 
+	 */
+	public int computeNumberOfPages(double fraction) {
+		if (fraction <= 0 || fraction > 1) {
+			throw new IllegalArgumentException("The fraction of memory to allocate must within (0, 1].");
+		}
+
+		return (int)(totalNumPages * fraction / numberOfSlots);
+	}
+
+	/**
+	 * Computes the memory size of the fraction per slot.
+	 *
+	 * @param fraction The fraction of the memory of the task slot.
+	 * @return The number of pages corresponding to the memory fraction.
+	 */
+	public long computeMemorySize(double fraction) {
+		return pageSize * computeNumberOfPages(fraction);
+	}
+
+	/**
+	 * Rounds the given value down to a multiple of the memory manager's page size.
+	 *
+	 * @return The given value, rounded down to a multiple of the page size.
+	 */
+	public long roundDownToPageSizeMultiple(long numBytes) {
+		return numBytes & roundingMask;
+	}
+	
+
+	// ------------------------------------------------------------------------
+	//  Memory Pools
+	// ------------------------------------------------------------------------
+
+	static abstract class MemoryPool {
+
+		abstract int getNumberOfAvailableMemorySegments();
+
+		abstract MemorySegment allocateNewSegment(Object owner);
+
+		abstract MemorySegment requestSegmentFromPool(Object owner);
+
+		abstract void returnSegmentToPool(MemorySegment segment);
+		
+		abstract void clear();
+	}
+
+	static final class HeapMemoryPool extends MemoryPool {
+
+		/** The collection of available memory segments */
+		private final ArrayDeque<byte[]> availableMemory;
+
+		private final int segmentSize;
+
+		public HeapMemoryPool(int numInitialSegments, int segmentSize) {
+			this.availableMemory = new ArrayDeque<byte[]>(numInitialSegments);
+			this.segmentSize = segmentSize;
+			
+			for (int i = 0; i < numInitialSegments; i++) {
+				this.availableMemory.add(new byte[segmentSize]);
+			}
+		}
+
+		@Override
+		HeapMemorySegment allocateNewSegment(Object owner) {
+			return HeapMemorySegment.FACTORY.allocateUnpooledSegment(segmentSize, owner);
+		}
+
+		@Override
+		HeapMemorySegment requestSegmentFromPool(Object owner) {
+			byte[] buf = availableMemory.remove();
+			return  HeapMemorySegment.FACTORY.wrapPooledHeapMemory(buf, owner);
+		}
+
+		@Override
+		void returnSegmentToPool(MemorySegment segment) {
+			if (segment.getClass() == HeapMemorySegment.class) {
+				HeapMemorySegment heapSegment = (HeapMemorySegment) segment;
+				availableMemory.add(heapSegment.getArray());
+				heapSegment.free();
+			}
+			else {
+				throw new IllegalArgumentException("Memory segment is not a " + HeapMemorySegment.class.getSimpleName());
+			}
+		}
+
+		@Override
+		protected int getNumberOfAvailableMemorySegments() {
+			return availableMemory.size();
+		}
+
+		@Override
+		void clear() {
+			availableMemory.clear();
+		}
+	}
+	
+	static final class HybridOffHeapMemoryPool extends MemoryPool {
+
+		/** The collection of available memory segments */
+		private final ArrayDeque<ByteBuffer> availableMemory;
+
+		private final int segmentSize;
+
+		public HybridOffHeapMemoryPool(int numInitialSegments, int segmentSize) {
+			this.availableMemory = new ArrayDeque<ByteBuffer>(numInitialSegments);
+			this.segmentSize = segmentSize;
+
+			for (int i = 0; i < numInitialSegments; i++) {
+				this.availableMemory.add(ByteBuffer.allocateDirect(segmentSize));
+			}
+		}
+
+		@Override
+		HybridMemorySegment allocateNewSegment(Object owner) {
+			ByteBuffer memory = ByteBuffer.allocateDirect(segmentSize);
+			return HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(memory, owner);
+		}
+
+		@Override
+		HybridMemorySegment requestSegmentFromPool(Object owner) {
+			ByteBuffer buf = availableMemory.remove();
+			return HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(buf, owner);
+		}
+
+		@Override
+		void returnSegmentToPool(MemorySegment segment) {
+			if (segment.getClass() == HybridMemorySegment.class) {
+				HybridMemorySegment hybridSegment = (HybridMemorySegment) segment;
+				ByteBuffer buf = hybridSegment.getOffHeapBuffer();
+				availableMemory.add(buf);
+				hybridSegment.free();
+			}
+			else {
+				throw new IllegalArgumentException("Memory segment is not a " + HeapMemorySegment.class.getSimpleName());
+			}
+		}
+
+		@Override
+		protected int getNumberOfAvailableMemorySegments() {
+			return availableMemory.size();
+		}
+
+		@Override
+		void clear() {
+			availableMemory.clear();
+		}
+	}
+}