You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by fh...@apache.org on 2015/10/26 22:50:44 UTC

[1/8] flink git commit: [FLINK-2890] Port StringSerializationSpeedBenchmark to JMH. [FLINK-2889] Port LongSerializationSpeedBenchmark to JMH.

Repository: flink
Updated Branches:
  refs/heads/master daa357aca -> 6a8e90b36


http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/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
deleted file mode 100644
index d898dee..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegmentOutView.java
+++ /dev/null
@@ -1,359 +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.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/75a52574/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
deleted file mode 100644
index 2163eb5..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/StringSerializationSpeedBenchmark.java
+++ /dev/null
@@ -1,207 +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.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;
-	}
-}


[2/8] flink git commit: [FLINK-2890] Port StringSerializationSpeedBenchmark to JMH. [FLINK-2889] Port LongSerializationSpeedBenchmark to JMH.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/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
deleted file mode 100644
index e247eed..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegment.java
+++ /dev/null
@@ -1,466 +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.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/75a52574/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
deleted file mode 100644
index 1e3b89e..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHeapMemorySegmentOutView.java
+++ /dev/null
@@ -1,359 +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.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();
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/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
deleted file mode 100644
index 57817b9..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegment.java
+++ /dev/null
@@ -1,887 +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.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/75a52574/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
deleted file mode 100644
index cda48e1..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureHybridMemorySegmentOutView.java
+++ /dev/null
@@ -1,359 +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.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/75a52574/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
deleted file mode 100644
index 1280242..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/PureOffHeapMemorySegment.java
+++ /dev/null
@@ -1,790 +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.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);
-		}
-	}
-}


[4/8] flink git commit: [FLINK-2890] Port StringSerializationSpeedBenchmark to JMH. [FLINK-2889] Port LongSerializationSpeedBenchmark to JMH.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHeapMemorySegment.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHeapMemorySegment.java
new file mode 100644
index 0000000..f36fcd9
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHeapMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHeapMemorySegmentOutView.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHeapMemorySegmentOutView.java
new file mode 100644
index 0000000..70d40d5
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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();
+		}
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHybridMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHybridMemorySegment.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHybridMemorySegment.java
new file mode 100644
index 0000000..05c3889
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHybridMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHybridMemorySegmentOutView.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureHybridMemorySegmentOutView.java
new file mode 100644
index 0000000..65bfe6b
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureOffHeapMemorySegment.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureOffHeapMemorySegment.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureOffHeapMemorySegment.java
new file mode 100644
index 0000000..7032c5e
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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);
+		}
+	}
+}


[7/8] flink git commit: [FLINK-2827] Close FileInputStream through try-with-resources to avoid unused open stream.

Posted by fh...@apache.org.
[FLINK-2827] Close FileInputStream through try-with-resources to avoid unused open stream.

This closes #1276


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

Branch: refs/heads/master
Commit: 6a8e90b3621bb96304b325a4ae8f7f5575ec909a
Parents: 7265d81
Author: Saumitra Shahapure <sa...@gmail.com>
Authored: Tue Oct 20 19:28:31 2015 +0200
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 26 20:05:03 2015 +0100

----------------------------------------------------------------------
 .../apache/flink/streaming/connectors/twitter/TwitterSource.java | 4 +---
 1 file changed, 1 insertion(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6a8e90b3/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
----------------------------------------------------------------------
diff --git a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
index bad0f8c..d290f98 100644
--- a/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
+++ b/flink-streaming-connectors/flink-connector-twitter/src/main/java/org/apache/flink/streaming/connectors/twitter/TwitterSource.java
@@ -133,10 +133,8 @@ public class TwitterSource extends RichSourceFunction<String> {
 	private Properties loadAuthenticationProperties() {
 		
 		Properties properties = new Properties();
-		try {
-			InputStream input = new FileInputStream(authPath);
+		try (InputStream input = new FileInputStream(authPath)) {
 			properties.load(input);
-			input.close();
 		} catch (Exception e) {
 			throw new RuntimeException("Cannot open .properties file: " + authPath, e);
 		}


[5/8] flink git commit: [FLINK-2890] Port StringSerializationSpeedBenchmark to JMH. [FLINK-2889] Port LongSerializationSpeedBenchmark to JMH.

Posted by fh...@apache.org.
[FLINK-2890] Port StringSerializationSpeedBenchmark to JMH.
[FLINK-2889] Port LongSerializationSpeedBenchmark to JMH.

This closes #1284
This closes #1283


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

Branch: refs/heads/master
Commit: 75a5257412606ac70113850439457cce7da3b2e6
Parents: daa357a
Author: gallenvara <ga...@126.com>
Authored: Thu Oct 22 14:35:11 2015 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 26 20:04:38 2015 +0100

----------------------------------------------------------------------
 flink-benchmark/pom.xml                         |    9 +-
 .../memory/LongSerializationSpeedBenchmark.java |  225 +++
 .../StringSerializationSpeedBenchmark.java      |  209 +++
 .../segments/CoreMemorySegmentOutView.java      |  360 ++++
 .../segments/MemorySegmentSpeedBenchmark.java   | 1633 ++++++++++++++++++
 .../memory/segments/PureHeapMemorySegment.java  |  466 +++++
 .../segments/PureHeapMemorySegmentOutView.java  |  359 ++++
 .../segments/PureHybridMemorySegment.java       |  887 ++++++++++
 .../PureHybridMemorySegmentOutView.java         |  359 ++++
 .../segments/PureOffHeapMemorySegment.java      |  790 +++++++++
 .../PureOffHeapMemorySegmentOutView.java        |  359 ++++
 .../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 ---
 21 files changed, 5655 insertions(+), 5653 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/flink-benchmark/pom.xml
----------------------------------------------------------------------
diff --git a/flink-benchmark/pom.xml b/flink-benchmark/pom.xml
index 281273f..d6ba1d7 100644
--- a/flink-benchmark/pom.xml
+++ b/flink-benchmark/pom.xml
@@ -36,7 +36,7 @@ under the License.
 
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <jmh.version>1.4.1</jmh.version>
+    <jmh.version>1.11</jmh.version>
     <uberjar.name>benchmarks</uberjar.name>
   </properties>
 
@@ -57,6 +57,13 @@ under the License.
       <version>${jmh.version}</version>
       <scope>provided</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-core</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>  
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/LongSerializationSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/LongSerializationSpeedBenchmark.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/LongSerializationSpeedBenchmark.java
new file mode 100644
index 0000000..3e2605a
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/LongSerializationSpeedBenchmark.java
@@ -0,0 +1,225 @@
+/*
+ * 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.benchmark.core.memory;
+
+import org.apache.flink.api.common.typeutils.base.LongSerializer;
+import org.apache.flink.benchmark.core.memory.segments.CoreMemorySegmentOutView;
+import org.apache.flink.benchmark.core.memory.segments.PureHeapMemorySegment;
+import org.apache.flink.benchmark.core.memory.segments.PureHeapMemorySegmentOutView;
+import org.apache.flink.benchmark.core.memory.segments.PureHybridMemorySegment;
+import org.apache.flink.benchmark.core.memory.segments.PureHybridMemorySegmentOutView;
+import org.apache.flink.core.memory.HeapMemorySegment;
+import org.apache.flink.core.memory.HybridMemorySegment;
+import org.apache.flink.core.memory.MemorySegment;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class LongSerializationSpeedBenchmark {
+	
+	private final static int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
+	
+	private final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
+	
+	private final static long innerRounds = LARGE_SEGMENT_SIZE / 8;
+	
+	private final static int outerRounds = 10;
+	
+	private MemorySegment coreHeap;
+	
+	private MemorySegment coreHybridOnHeap;
+	
+	private MemorySegment coreHybridOffHeap;
+	
+	private PureHeapMemorySegment pureHeap;
+	
+	private PureHybridMemorySegment pureHybridOnHeap;
+	
+	private PureHybridMemorySegment pureHybridOffHeap;
+	
+	private LongSerializer ser;
+	
+	
+	@Setup
+	public void init() {
+		final ByteBuffer largeOffHeap = ByteBuffer.allocateDirect(LARGE_SEGMENT_SIZE);
+		
+		fillOnHeap(largeSegment, (byte) -1);
+		fillOffHeap(largeOffHeap, (byte) -1);
+		
+		this.coreHeap = HeapMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null);
+		this.coreHybridOnHeap = HybridMemorySegment.FACTORY.wrapPooledHeapMemory(largeSegment, null);
+		this.coreHybridOffHeap = HybridMemorySegment.FACTORY.wrapPooledOffHeapMemory(largeOffHeap, null);
+		this.pureHeap = new PureHeapMemorySegment(largeSegment);
+		this.pureHybridOnHeap = new PureHybridMemorySegment(largeSegment);
+		this.pureHybridOffHeap = new PureHybridMemorySegment(largeOffHeap);
+		this.ser = LongSerializer.INSTANCE;
+	}
+	
+	@Benchmark
+	public void coreHeapMemorySegment() throws Exception {
+		
+		ArrayList<MemorySegment> memory = new ArrayList<>();
+		memory.add(coreHeap);
+		ArrayList<MemorySegment> target = new ArrayList<>();
+		
+		CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+		
+		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();
+		}
+	}
+	
+	@Benchmark
+	public void coreHybridOnHeapMemorySegment() throws Exception {
+		
+		ArrayList<MemorySegment> memory = new ArrayList<>();
+		memory.add(coreHybridOnHeap);
+		ArrayList<MemorySegment> target = new ArrayList<>();
+		
+		CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+		
+		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();
+		}
+	}
+	
+	@Benchmark
+	public void coreHybridOffHeapMemorySegment() throws Exception {
+		
+		ArrayList<MemorySegment> memory = new ArrayList<>();
+		memory.add(coreHybridOffHeap);
+		ArrayList<MemorySegment> target = new ArrayList<>();
+		
+		CoreMemorySegmentOutView output = new CoreMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+		
+		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();
+		}
+	}
+	
+	@Benchmark
+	public void pureHeapMemorySegment() throws Exception {
+		
+		ArrayList<PureHeapMemorySegment> memory = new ArrayList<>();
+		memory.add(pureHeap);
+		ArrayList<PureHeapMemorySegment> target = new ArrayList<>();
+		
+		PureHeapMemorySegmentOutView output = new PureHeapMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+		
+		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();
+		}
+	}
+	
+	@Benchmark
+	public void pureHybridOnHeapMemorySegment() throws Exception {
+		
+		ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+		memory.add(pureHybridOnHeap);
+		ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+		
+		PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+		
+		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();
+		}
+	}
+	
+	@Benchmark
+	public void pureHybridOffHeapMemorySegment() throws Exception {
+		
+		ArrayList<PureHybridMemorySegment> memory = new ArrayList<>();
+		memory.add(pureHybridOffHeap);
+		ArrayList<PureHybridMemorySegment> target = new ArrayList<>();
+		
+		PureHybridMemorySegmentOutView output = new PureHybridMemorySegmentOutView(memory, target, LARGE_SEGMENT_SIZE);
+		
+		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();
+		}
+	}
+	
+	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);
+		}
+	}
+	
+	public static void main(String[] args) throws Exception {
+		Options opt = new OptionsBuilder()
+				.include(LongSerializationSpeedBenchmark.class.getSimpleName())
+				.warmupIterations(2)
+				.measurementIterations(2)
+				.forks(1)
+				.build();
+		new Runner(opt).run();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/StringSerializationSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/StringSerializationSpeedBenchmark.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/StringSerializationSpeedBenchmark.java
new file mode 100644
index 0000000..d1268a7
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/StringSerializationSpeedBenchmark.java
@@ -0,0 +1,209 @@
+/*
+ * 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.benchmark.core.memory;
+
+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 org.apache.flink.benchmark.core.memory.segments.*;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Thread)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class StringSerializationSpeedBenchmark {
+	
+	private final int LARGE_SEGMENT_SIZE = 1024 * 1024 * 1024;
+	
+	private final byte[] largeSegment = new byte[LARGE_SEGMENT_SIZE];
+	
+	private final int outerRounds = 10;
+	
+	private final int innerRounds = 5000;
+	
+	private ByteBuffer largeOffHeap;
+	
+	private String[] randomStrings;
+	
+	private StringSerializer ser;
+	
+	@Setup
+	public void init() throws Exception {
+		
+		this.largeOffHeap = ByteBuffer.allocateDirect(LARGE_SEGMENT_SIZE);
+		this.randomStrings = generateRandomStrings(5468917685263896L, 1000, 128, 6, true);
+		this.ser = StringSerializer.INSTANCE;
+		
+	}
+	
+	@Benchmark
+	public void coreHeapMemorySegment() throws Exception {
+		
+		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);
+				}
+			}
+		}
+	}
+	
+	@Benchmark
+	public void coreHybridMemorySegmentOnHeap() throws Exception {
+		
+		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);
+				}
+			}
+		}
+	}
+	
+	@Benchmark
+	public void coreHybridMemorySegmentOffHeap() throws Exception {
+		
+		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);
+				}
+			}
+		}
+	}
+	
+	@Benchmark
+	public void pureHybridMemorySegmentOnheap() throws Exception {
+		
+		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);
+				}
+			}
+		}
+	}
+	
+	@Benchmark
+	public void pureHybridMemorySegmentOffHeap() throws Exception {
+		
+		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);
+				}
+			}
+		}
+	}
+	
+	@Benchmark
+	public void pureHeapMemorySegment() throws Exception {
+		
+		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);
+				}
+			}
+		}
+	}
+	
+	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;
+	}
+	
+	public static void main(String[] args) throws Exception {
+		Options opt = new OptionsBuilder()
+				.include(StringSerializationSpeedBenchmark.class.getSimpleName())
+				.warmupIterations(2)
+				.measurementIterations(2)
+				.forks(1)
+				.build();
+		new Runner(opt).run();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/CoreMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/CoreMemorySegmentOutView.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/CoreMemorySegmentOutView.java
new file mode 100644
index 0000000..58c42ac
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/MemorySegmentSpeedBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/MemorySegmentSpeedBenchmark.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/MemorySegmentSpeedBenchmark.java
new file mode 100644
index 0000000..111796b
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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();
+	}
+}


[3/8] flink git commit: [FLINK-2890] Port StringSerializationSpeedBenchmark to JMH. [FLINK-2889] Port LongSerializationSpeedBenchmark to JMH.

Posted by fh...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureOffHeapMemorySegmentOutView.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureOffHeapMemorySegmentOutView.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/PureOffHeapMemorySegmentOutView.java
new file mode 100644
index 0000000..424a190
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/core/memory/segments/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.benchmark.core.memory.segments;
+
+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/75a52574/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
deleted file mode 100644
index df3f8be..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/CoreMemorySegmentOutView.java
+++ /dev/null
@@ -1,360 +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.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/75a52574/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
deleted file mode 100644
index ed3c1f5..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/LongSerializationSpeedBenchmark.java
+++ /dev/null
@@ -1,232 +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.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);
-		}
-	}
-}

http://git-wip-us.apache.org/repos/asf/flink/blob/75a52574/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
deleted file mode 100644
index 454c821..0000000
--- a/flink-core/src/test/java/org/apache/flink/core/memory/benchmarks/MemorySegmentSpeedBenchmark.java
+++ /dev/null
@@ -1,1633 +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.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();
-	}
-}


[6/8] flink git commit: [FLINK-2853] Port MutableHashTablePerformanceBenchmark to JMH.

Posted by fh...@apache.org.
[FLINK-2853] Port MutableHashTablePerformanceBenchmark to JMH.

This closes #1267


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

Branch: refs/heads/master
Commit: e5ee55bdaf0dcd6e3dc38b52964a8570b22dd671
Parents: 75a5257
Author: gallenvara <ga...@126.com>
Authored: Fri Oct 16 16:48:30 2015 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 26 20:05:03 2015 +0100

----------------------------------------------------------------------
 flink-benchmark/pom.xml                         |  11 +-
 .../MutableHashTablePerformanceBenchmark.java   | 359 +++++++++++++++++++
 .../MutableHashTablePerformanceBenchmark.java   | 262 --------------
 3 files changed, 368 insertions(+), 264 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/e5ee55bd/flink-benchmark/pom.xml
----------------------------------------------------------------------
diff --git a/flink-benchmark/pom.xml b/flink-benchmark/pom.xml
index d6ba1d7..b22e0d1 100644
--- a/flink-benchmark/pom.xml
+++ b/flink-benchmark/pom.xml
@@ -56,14 +56,21 @@ under the License.
       <artifactId>jmh-generator-annprocess</artifactId>
       <version>${jmh.version}</version>
       <scope>provided</scope>
+    </dependency>  
+    <dependency>
+      <groupId>org.apache.flink</groupId>
+      <artifactId>flink-runtime</artifactId>
+      <version>0.10-SNAPSHOT</version>
+      <type>test-jar</type>
+      <scope>test</scope>
     </dependency>
     <dependency>
       <groupId>org.apache.flink</groupId>
-      <artifactId>flink-core</artifactId>
+      <artifactId>flink-java</artifactId>
       <version>${project.version}</version>
       <type>test-jar</type>
       <scope>test</scope>
-    </dependency>  
+    </dependency>
   </dependencies>
 
   <build>

http://git-wip-us.apache.org/repos/asf/flink/blob/e5ee55bd/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
new file mode 100644
index 0000000..186c595
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/runtime/operators/hash/MutableHashTablePerformanceBenchmark.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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.benchmark.runtime.operators.hash;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+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.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.memory.MemoryAllocationException;
+import org.apache.flink.runtime.memory.MemoryManager;
+import org.apache.flink.runtime.operators.hash.MutableHashTable;
+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;
+import org.apache.flink.runtime.operators.testutils.types.StringPairPairComparator;
+import org.apache.flink.runtime.operators.testutils.types.StringPairSerializer;
+import org.apache.flink.util.MutableObjectIterator;
+
+import org.junit.Assert;
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import static org.junit.Assert.fail;
+
+@State(Scope.Thread)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class MutableHashTablePerformanceBenchmark {
+	
+	private static final AbstractInvokable MEM_OWNER = new DummyInvokable();
+	
+	private MemoryManager memManager;
+	private IOManager ioManager;
+	
+	private TypeSerializer<StringPair> pairBuildSideAccesssor;
+	private TypeSerializer<StringPair> pairProbeSideAccesssor;
+	private TypeComparator<StringPair> pairBuildSideComparator;
+	private TypeComparator<StringPair> pairProbeSideComparator;
+	private TypePairComparator<StringPair, StringPair> pairComparator;
+	
+	private static final String COMMENT = "this comments should contains a 96 byte data, 100 plus another integer value and seperator char.";
+	
+	@Setup
+	public void setup() {
+		this.pairBuildSideAccesssor = new StringPairSerializer();
+		this.pairProbeSideAccesssor = new StringPairSerializer();
+		this.pairBuildSideComparator = new StringPairComparator();
+		this.pairProbeSideComparator = new StringPairComparator();
+		this.pairComparator = new StringPairPairComparator();
+		
+		this.memManager = new MemoryManager(64 * 1024 * 1024, 1);
+		this.ioManager = new IOManagerAsync();
+	}
+	
+	@TearDown
+	public void tearDown() {
+		// shut down I/O manager and Memory Manager and verify the correct shutdown
+		this.ioManager.shutdown();
+		if (!this.ioManager.isProperlyShutDown()) {
+			fail("I/O manager was not property shut down.");
+		}
+		if (!this.memManager.verifyEmpty()) {
+			fail("Not all memory was properly released to the memory manager --> Memory Leak.");
+		}
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithBloomFilter1() throws IOException {
+		// ----------------------------------------------90% filtered during probe spill phase-----------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 10000000] with step of 10 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 10;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = 500000;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
+		
+		System.out.println("HybridHashJoin1:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithoutBloomFilter1() throws IOException {
+		// ----------------------------------------------90% filtered during probe spill phase-----------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 10000000] with step of 10 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 10;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = 500000;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
+		
+		System.out.println("HybridHashJoin1:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithBloomFilter2() throws IOException {
+		// ----------------------------------------------80% filtered during probe spill phase-----------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 5000000] with step of 5 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 5;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = 1000000;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
+		
+		System.out.println("HybridHashJoin2:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithoutBloomFilter2() throws IOException {
+		// ----------------------------------------------80% filtered during probe spill phase-----------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 5000000] with step of 5 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 5;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = 1000000;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
+		
+		System.out.println("HybridHashJoin2:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithBloomFilter3() throws IOException {
+		// ----------------------------------------------50% filtered during probe spill phase-------------------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 2000000] with step of 2 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 2;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = 2500000;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
+		
+		System.out.println("HybridHashJoin3:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithoutBloomFilter3() throws IOException {
+		// ----------------------------------------------50% filtered during probe spill phase-------------------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 2000000] with step of 2 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 2;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = 2500000;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
+		
+		System.out.println("HybridHashJoin3:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithBloomFilter4() throws IOException {
+		// ----------------------------------------------0% filtered during probe spill phase-----------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 1000000] with step of 1 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 1;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = probeSize / buildStep;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
+		
+		System.out.println("HybridHashJoin4:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	@Benchmark
+	public void compareMutableHashTableWithoutBloomFilter4() throws IOException {
+		// ----------------------------------------------0% filtered during probe spill phase-----------------------------------------
+		// create a build input with 1000000 records with key spread between [0 -- 1000000] with step of 1 for nearby records.
+		int buildSize = 1000000;
+		int buildStep = 1;
+		int buildScope = buildStep * buildSize;
+		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
+		int probeSize = 5000000;
+		int probeStep = 1;
+		int probeScope = buildSize;
+		
+		int expectedResult = probeSize / buildStep;
+		
+		this.hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
+		
+		System.out.println("HybridHashJoin4:");
+		System.out.println("Build input size: " + 100 * buildSize);
+		System.out.println("Probe input size: " + 100 * probeSize);
+		System.out.println("Available memory: " + this.memManager.getMemorySize());
+		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
+	}
+	
+	private long hybridHashJoin(int buildSize, int buildStep, int buildScope, int probeSize,
+		int probeStep, int probeScope, int expectedResultSize, boolean enableBloomFilter) throws IOException {
+		
+		InputIterator buildIterator = new InputIterator(buildSize, buildStep, buildScope);
+		InputIterator probeIterator = new InputIterator(probeSize, probeStep, probeScope);
+		
+		// allocate the memory for the HashTable
+		List<MemorySegment> memSegments;
+		try {
+			// 33 is minimum number of pages required to perform hash join this inputs
+			memSegments = this.memManager.allocatePages(MEM_OWNER, (int) (this.memManager.getMemorySize() / this.memManager.getPageSize()));
+		} catch (MemoryAllocationException maex) {
+			fail("Memory for the Join could not be provided.");
+			return -1;
+		}
+		
+		// ----------------------------------------------------------------------------------------
+		
+		final MutableHashTable<StringPair, StringPair> join = new MutableHashTable<StringPair, StringPair>(
+			this.pairBuildSideAccesssor, this.pairProbeSideAccesssor,
+			this.pairBuildSideComparator, this.pairProbeSideComparator, this.pairComparator,
+			memSegments, ioManager, enableBloomFilter);
+		join.open(buildIterator, probeIterator);
+		
+		final StringPair recordReuse = new StringPair();
+		int numRecordsInJoinResult = 0;
+		
+		while (join.nextRecord()) {
+			MutableHashTable.HashBucketIterator<StringPair, StringPair> buildSide = join.getBuildSideIterator();
+			while (buildSide.next(recordReuse) != null) {
+				numRecordsInJoinResult++;
+			}
+		}
+		Assert.assertEquals("Wrong number of records in join result.", expectedResultSize, numRecordsInJoinResult);
+		
+		join.close();
+		// ----------------------------------------------------------------------------------------
+		
+		this.memManager.release(join.getFreedMemory());
+		return 1;
+	}
+	
+	
+	static class InputIterator implements MutableObjectIterator<StringPair> {
+		
+		private int numLeft;
+		private int distance;
+		private int scope;
+		
+		public InputIterator(int size, int distance, int scope) {
+			this.numLeft = size;
+			this.distance = distance;
+			this.scope = scope;
+		}
+		
+		@Override
+		public StringPair next(StringPair reuse) throws IOException {
+			if (this.numLeft > 0) {
+				numLeft--;
+				int currentKey = (numLeft * distance) % scope;
+				reuse.setKey(Integer.toString(currentKey));
+				reuse.setValue(COMMENT);
+				return reuse;
+			} else {
+				return null;
+			}
+		}
+		
+		@Override
+		public StringPair next() throws IOException {
+			return next(new StringPair());
+		}
+	}
+	
+	public static void main(String[] args) throws Exception {
+		Options opt = new OptionsBuilder()
+				.include(MutableHashTablePerformanceBenchmark.class.getSimpleName())
+				.warmupIterations(2)
+				.measurementIterations(2)
+				.forks(1)
+				.build();
+		new Runner(opt).run();
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/e5ee55bd/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
deleted file mode 100644
index 70c9427..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/MutableHashTablePerformanceBenchmark.java
+++ /dev/null
@@ -1,262 +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
- * <p/>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p/>
- * 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.operators.hash;
-
-import java.io.IOException;
-import java.util.List;
-
-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.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.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;
-import org.apache.flink.runtime.operators.testutils.types.StringPairPairComparator;
-import org.apache.flink.runtime.operators.testutils.types.StringPairSerializer;
-import org.apache.flink.util.MutableObjectIterator;
-
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import static org.junit.Assert.fail;
-
-public class MutableHashTablePerformanceBenchmark {
-	
-	private static final AbstractInvokable MEM_OWNER = new DummyInvokable();
-	
-	private MemoryManager memManager;
-	private IOManager ioManager;
-	
-	private TypeSerializer<StringPair> pairBuildSideAccesssor;
-	private TypeSerializer<StringPair> pairProbeSideAccesssor;
-	private TypeComparator<StringPair> pairBuildSideComparator;
-	private TypeComparator<StringPair> pairProbeSideComparator;
-	private TypePairComparator<StringPair, StringPair> pairComparator;
-	
-	private static final String COMMENT = "this comments should contains a 96 byte data, 100 plus another integer value and seperator char.";
-	
-	
-	@Before
-	public void setup() {
-		this.pairBuildSideAccesssor = new StringPairSerializer();
-		this.pairProbeSideAccesssor = new StringPairSerializer();
-		this.pairBuildSideComparator = new StringPairComparator();
-		this.pairProbeSideComparator = new StringPairComparator();
-		this.pairComparator = new StringPairPairComparator();
-		
-		this.memManager = new MemoryManager(64 * 1024 * 1024, 1);
-		this.ioManager = new IOManagerAsync();
-	}
-	
-	@After
-	public void tearDown() {
-		// shut down I/O manager and Memory Manager and verify the correct shutdown
-		this.ioManager.shutdown();
-		if (!this.ioManager.isProperlyShutDown()) {
-			fail("I/O manager was not property shut down.");
-		}
-		if (!this.memManager.verifyEmpty()) {
-			fail("Not all memory was properly released to the memory manager --> Memory Leak.");
-		}
-	}
-	
-	@Test
-	public void compareMutableHashTablePerformance1() throws IOException {
-		// ----------------------------------------------90% filtered during probe spill phase-----------------------------------------
-		// create a build input with 1000000 records with key spread between [0 -- 10000000] with step of 10 for nearby records.
-		int buildSize = 1000000;
-		int buildStep = 10;
-		int buildScope = buildStep * buildSize;
-		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
-		int probeSize = 5000000;
-		int probeStep = 1;
-		int probeScope = buildSize;
-		
-		int expectedResult = 500000;
-		
-		long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
-		long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
-		
-		System.out.println("HybridHashJoin2:");
-		System.out.println("Build input size: " + 100 * buildSize);
-		System.out.println("Probe input size: " + 100 * probeSize);
-		System.out.println("Available memory: " + this.memManager.getMemorySize());
-		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
-		System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost));
-	}
-	
-	@Test
-	public void compareMutableHashTablePerformance2() throws IOException {
-		// ----------------------------------------------80% filtered during probe spill phase-----------------------------------------
-		// create a build input with 1000000 records with key spread between [0 -- 5000000] with step of 5 for nearby records.
-		int buildSize = 1000000;
-		int buildStep = 5;
-		int buildScope = buildStep * buildSize;
-		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
-		int probeSize = 5000000;
-		int probeStep = 1;
-		int probeScope = buildSize;
-		
-		int expectedResult = 1000000;
-		
-		long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
-		long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
-		
-		System.out.println("HybridHashJoin3:");
-		System.out.println("Build input size: " + 100 * buildSize);
-		System.out.println("Probe input size: " + 100 * probeSize);
-		System.out.println("Available memory: " + this.memManager.getMemorySize());
-		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
-		System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost));
-	}
-	
-	@Test
-	public void compareMutableHashTablePerformance3() throws IOException {
-		// ----------------------------------------------50% filtered during probe spill phase-------------------------------------------------
-		// create a build input with 1000000 records with key spread between [0 -- 2000000] with step of 2 for nearby records.
-		int buildSize = 1000000;
-		int buildStep = 2;
-		int buildScope = buildStep * buildSize;
-		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
-		int probeSize = 5000000;
-		int probeStep = 1;
-		int probeScope = buildSize;
-		
-		int expectedResult = 2500000;
-		
-		long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
-		long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
-		
-		System.out.println("HybridHashJoin4:");
-		System.out.println("Build input size: " + 100 * buildSize);
-		System.out.println("Probe input size: " + 100 * probeSize);
-		System.out.println("Available memory: " + this.memManager.getMemorySize());
-		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
-		System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost));
-	}
-	
-	@Test
-	public void compareMutableHashTablePerformance4() throws IOException {
-		// ----------------------------------------------0% filtered during probe spill phase-----------------------------------------
-		// create a build input with 1000000 records with key spread between [0 -- 1000000] with step of 1 for nearby records.
-		int buildSize = 1000000;
-		int buildStep = 1;
-		int buildScope = buildStep * buildSize;
-		// create a probe input with 5000000 records with key spread between [0 -- 1000000] with distance of 1 for nearby records.
-		int probeSize = 5000000;
-		int probeStep = 1;
-		int probeScope = buildSize;
-		
-		int expectedResult = probeSize / buildStep;
-		
-		long withBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, true);
-		long withoutBloomFilterCost = hybridHashJoin(buildSize, buildStep, buildScope, probeSize, probeStep, probeScope, expectedResult, false);
-		
-		System.out.println("HybridHashJoin5:");
-		System.out.println("Build input size: " + 100 * buildSize);
-		System.out.println("Probe input size: " + 100 * probeSize);
-		System.out.println("Available memory: " + this.memManager.getMemorySize());
-		System.out.println("Probe record be filtered before spill: " + (1 - (double)probeScope / buildScope) * 100 + "% percent.");
-		System.out.println(String.format("Cost: without bloom filter(%d), with bloom filter(%d)", withoutBloomFilterCost, withBloomFilterCost));
-	}
-	
-	private long hybridHashJoin(int buildSize, int buildStep, int buildScope, int probeSize,
-		int probeStep, int probeScope, int expectedResultSize, boolean enableBloomFilter) throws IOException {
-		
-		InputIterator buildIterator = new InputIterator(buildSize, buildStep, buildScope);
-		InputIterator probeIterator = new InputIterator(probeSize, probeStep, probeScope);
-		
-		// allocate the memory for the HashTable
-		List<MemorySegment> memSegments;
-		try {
-			// 33 is minimum number of pages required to perform hash join this inputs
-			memSegments = this.memManager.allocatePages(MEM_OWNER, (int) (this.memManager.getMemorySize() / this.memManager.getPageSize()));
-		} catch (MemoryAllocationException maex) {
-			fail("Memory for the Join could not be provided.");
-			return -1;
-		}
-		
-		// ----------------------------------------------------------------------------------------
-		
-		long start = System.currentTimeMillis();
-		final MutableHashTable<StringPair, StringPair> join = new MutableHashTable<StringPair, StringPair>(
-			this.pairBuildSideAccesssor, this.pairProbeSideAccesssor,
-			this.pairBuildSideComparator, this.pairProbeSideComparator, this.pairComparator,
-			memSegments, ioManager, enableBloomFilter);
-		join.open(buildIterator, probeIterator);
-		
-		final StringPair recordReuse = new StringPair();
-		int numRecordsInJoinResult = 0;
-		
-		while (join.nextRecord()) {
-			MutableHashTable.HashBucketIterator<StringPair, StringPair> buildSide = join.getBuildSideIterator();
-			while (buildSide.next(recordReuse) != null) {
-				numRecordsInJoinResult++;
-			}
-		}
-		Assert.assertEquals("Wrong number of records in join result.", expectedResultSize, numRecordsInJoinResult);
-		
-		join.close();
-		long cost = System.currentTimeMillis() - start;
-		// ----------------------------------------------------------------------------------------
-		
-		this.memManager.release(join.getFreedMemory());
-		return cost;
-	}
-	
-	
-	static class InputIterator implements MutableObjectIterator<StringPair> {
-		
-		private int numLeft;
-		private int distance;
-		private int scope;
-		
-		public InputIterator(int size, int distance, int scope) {
-			this.numLeft = size;
-			this.distance = distance;
-			this.scope = scope;
-		}
-		
-		@Override
-		public StringPair next(StringPair reuse) throws IOException {
-			if (this.numLeft > 0) {
-				numLeft--;
-				int currentKey = (numLeft * distance) % scope;
-				reuse.setKey(Integer.toString(currentKey));
-				reuse.setValue(COMMENT);
-				return reuse;
-			} else {
-				return null;
-			}
-		}
-		
-		@Override
-		public StringPair next() throws IOException {
-			return next(new StringPair());
-		}
-	}
-}


[8/8] flink git commit: [FLINK-2919] Port FieldAccessMinibenchmark to JMH.

Posted by fh...@apache.org.
[FLINK-2919] Port FieldAccessMinibenchmark to JMH.

This closes #1300


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

Branch: refs/heads/master
Commit: 7265d81ff95aff4ddfbcbd4ef25869ea8f159769
Parents: e5ee55b
Author: gallenvara <ga...@126.com>
Authored: Mon Oct 26 15:15:14 2015 +0800
Committer: Fabian Hueske <fh...@apache.org>
Committed: Mon Oct 26 20:05:03 2015 +0100

----------------------------------------------------------------------
 .../runtime/FieldAccessMinibenchmark.java       | 170 +++++++++++++++++++
 .../runtime/FieldAccessMinibenchmark.java       | 150 ----------------
 2 files changed, 170 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/7265d81f/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/FieldAccessMinibenchmark.java
----------------------------------------------------------------------
diff --git a/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/FieldAccessMinibenchmark.java b/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/FieldAccessMinibenchmark.java
new file mode 100644
index 0000000..0a434e9
--- /dev/null
+++ b/flink-benchmark/src/test/java/org/apache/flink/benchmark/api/java/typeutils/runtime/FieldAccessMinibenchmark.java
@@ -0,0 +1,170 @@
+/*
+ * 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.benchmark.api.java.typeutils.runtime;
+
+import org.openjdk.jmh.annotations.*;
+import org.openjdk.jmh.results.RunResult;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.RunnerException;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.lang.reflect.Field;
+import java.util.Collection;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Thread)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class FieldAccessMinibenchmark {
+
+	private final long RUNS = 1000000000L;
+	static Field wordDescField;
+	static Field wordField;
+	static {
+		try {
+			wordDescField = WC.class.getField("wordDesc");
+			wordField = ComplexWordDescriptor.class.getField("word");
+		} catch (Exception e) {
+			e.printStackTrace();
+		}
+	}
+
+	@Setup
+	public void warmUp() throws NoSuchFieldException,IllegalAccessException{
+		WC word0 = new WC(14, "Hallo");
+		WC word1 = new WC(3, "Hola");
+		for (long i = 0; i < 100000000; i++) {
+			compareCodeGenPublicFields(word0, word1);
+			compareCodeGenMethods(word0, word1);
+			compareReflective(word0, word1);
+		}
+	}
+
+	public static class ComplexWordDescriptor {
+		public String word;
+
+		public String getWord() {
+			return word;
+		}
+	}
+
+	public static class WC {
+		public int count;
+		public ComplexWordDescriptor wordDesc;
+
+		public WC(int c, String s) throws NoSuchFieldException,
+				SecurityException {
+			this.count = c;
+			this.wordDesc = new ComplexWordDescriptor();
+			this.wordDesc.word = s;
+		}
+
+		public ComplexWordDescriptor getWordDesc() {
+			return wordDesc;
+		}
+
+	}
+
+	public static int compareCodeGenPublicFields(WC w1, WC w2) {
+		return w1.wordDesc.word.compareTo(w2.wordDesc.word);
+	}
+
+	public static int compareCodeGenMethods(WC w1, WC w2) {
+		return w1.getWordDesc().getWord().compareTo(w2.getWordDesc().getWord());
+	}
+
+	public static int compareReflective(WC w1, WC w2)
+			throws IllegalArgumentException, IllegalAccessException {
+		// get String of w1
+		Object wordDesc1 = wordDescField.get(w1);
+		String word2cmp1 = (String) wordField.get(wordDesc1);
+
+		// get String of w2
+		Object wordDesc2 = wordDescField.get(w2);
+		String word2cmp2 = (String) wordField.get(wordDesc2);
+
+		return word2cmp1.compareTo(word2cmp2);
+	}
+
+	@Benchmark
+	public void codeGenPublicFields() throws NoSuchFieldException {
+		WC word0 = new WC(14, "Hallo");
+		WC word1 = new WC(3, "Hola");
+		for (long i = 0; i < RUNS; i++) {
+			int a = compareCodeGenPublicFields(word0, word1);
+			if (a == 0) {
+				System.err.println("hah");
+			}
+		}
+	}
+
+	@Benchmark
+	public void codeGenMethods() throws NoSuchFieldException{
+		WC word0 = new WC(14, "Hallo");
+		WC word1 = new WC(3, "Hola");
+		for (long i = 0; i < RUNS; i++) {
+			int a = compareCodeGenPublicFields(word0, word1);
+			if (a == 0) {
+				System.err.println("hah");
+			}
+		}
+	}
+
+	@Benchmark
+	public void reflection() throws NoSuchFieldException,IllegalAccessException {
+		WC word0 = new WC(14, "Hallo");
+		WC word1 = new WC(3, "Hola");
+		for (long i = 0; i < RUNS; i++) {
+			int a = compareReflective(word0, word1);
+			if (a == 0) {
+				System.err.println("hah");
+			}
+		}
+	}
+
+	/**
+	 * results on Core i7 2600k
+	 *
+	 *
+	 * warming up Code gen 5019 Reflection 20364 Factor = 4.057382
+	 */
+	public static void main(String[] args) throws RunnerException {
+
+		Options opt = new OptionsBuilder()
+				.include(FieldAccessMinibenchmark.class.getSimpleName())
+				.warmupIterations(2)
+				.measurementIterations(2)
+				.forks(1)
+				.build();
+		Collection<RunResult> results = new Runner(opt).run();
+		double[] score = new double[3];
+		int count = 0;
+		final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
+		String jvm = bean.getVmName() + " - " + bean.getVmVendor() + " - "
+				+ bean.getSpecVersion() + '/' + bean.getVmVersion();
+		System.err.println("Jvm info : " + jvm);
+		for (RunResult r : results) {
+			score[count++] = r.getPrimaryResult().getScore();
+		}
+		System.err.println("Factor vs public = " + score[2] / score[1]);
+		System.err.println("Factor vs methods = " + score[2] / score[0]);
+	}
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/7265d81f/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/FieldAccessMinibenchmark.java
----------------------------------------------------------------------
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/FieldAccessMinibenchmark.java b/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/FieldAccessMinibenchmark.java
deleted file mode 100644
index 8a0ec82..0000000
--- a/flink-java/src/test/java/org/apache/flink/api/java/typeutils/runtime/FieldAccessMinibenchmark.java
+++ /dev/null
@@ -1,150 +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.api.java.typeutils.runtime;
-
-import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.lang.reflect.Field;
-
-public class FieldAccessMinibenchmark {
-
-	static Field wordDescField;
-	static Field wordField;
-	static {
-		try {
-			wordDescField = WC.class.getField("wordDesc");
-			wordField = ComplexWordDescriptor.class.getField("word");
-		} catch (Exception e) {
-			e.printStackTrace();
-		}
-	}
-
-	public static class ComplexWordDescriptor {
-		public String word;
-
-		public String getWord() {
-			return word;
-		}
-	}
-
-	public static class WC {
-		public int count;
-		public ComplexWordDescriptor wordDesc;
-
-		public WC(int c, String s) throws NoSuchFieldException,
-				SecurityException {
-			this.count = c;
-			this.wordDesc = new ComplexWordDescriptor();
-			this.wordDesc.word = s;
-		}
-
-		public ComplexWordDescriptor getWordDesc() {
-			return wordDesc;
-		}
-
-	}
-
-	public static int compareCodeGenPublicFields(WC w1, WC w2) {
-		return w1.wordDesc.word.compareTo(w2.wordDesc.word);
-	}
-
-	public static int compareCodeGenMethods(WC w1, WC w2) {
-		return w1.getWordDesc().getWord().compareTo(w2.getWordDesc().getWord());
-	}
-
-	public static int compareReflective(WC w1, WC w2)
-			throws IllegalArgumentException, IllegalAccessException {
-		// get String of w1
-		Object wordDesc1 = wordDescField.get(w1);
-		String word2cmp1 = (String) wordField.get(wordDesc1);
-
-		// get String of w2
-		Object wordDesc2 = wordDescField.get(w2);
-		String word2cmp2 = (String) wordField.get(wordDesc2);
-
-		return word2cmp1.compareTo(word2cmp2);
-	}
-
-	/**
-	 * results on Core i7 2600k
-	 * 
-	 * 
-	 * warming up Code gen 5019 Reflection 20364 Factor = 4.057382
-	 */
-	public static void main(String[] args) throws NoSuchFieldException,
-			SecurityException, IllegalArgumentException, IllegalAccessException {
-		final long RUNS = 1000000000L;
-
-		final RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
-		String jvm = bean.getVmName() + " - " + bean.getVmVendor() + " - "
-				+ bean.getSpecVersion() + '/' + bean.getVmVersion();
-		System.err.println("Jvm info : " + jvm);
-
-		WC word0 = new WC(14, "Hallo");
-		WC word1 = new WC(3, "Hola");
-
-		System.err.println("warming up");
-		for (long i = 0; i < 100000000; i++) {
-			compareCodeGenPublicFields(word0, word1);
-			compareCodeGenMethods(word0, word1);
-			compareReflective(word0, word1);
-		}
-
-		System.err.println("Code gen public fields");
-		long startTime = System.currentTimeMillis();
-		for (long i = 0; i < RUNS; i++) {
-			int a = compareCodeGenPublicFields(word0, word1);
-			if (a == 0) {
-				System.err.println("hah");
-			}
-		}
-		long stopTime = System.currentTimeMillis();
-		long elapsedTimeGen = stopTime - startTime;
-		System.err.println(elapsedTimeGen);
-
-		System.err.println("Code gen methods");
-		startTime = System.currentTimeMillis();
-		for (long i = 0; i < RUNS; i++) {
-			int a = compareCodeGenPublicFields(word0, word1);
-			if (a == 0) {
-				System.err.println("hah");
-			}
-		}
-		stopTime = System.currentTimeMillis();
-		long elapsedTimeGenMethods = stopTime - startTime;
-		System.err.println(elapsedTimeGenMethods);
-
-		System.err.println("Reflection");
-
-		startTime = System.currentTimeMillis();
-		for (long i = 0; i < RUNS; i++) {
-			int a = compareReflective(word0, word1);
-			if (a == 0) {
-				System.err.println("hah");
-			}
-		}
-		stopTime = System.currentTimeMillis();
-		long elapsedTimeRef = stopTime - startTime;
-		System.err.println(elapsedTimeRef);
-
-		System.err.println("Factor vs public = "
-				+ (elapsedTimeRef / (float) elapsedTimeGen));
-		System.err.println("Factor vs methods = "
-				+ (elapsedTimeRef / (float) elapsedTimeGenMethods));
-	}
-}